From f3d9b819f3c013cd402ed98d01842173c45a5dd6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 8 May 2024 00:02:44 -0700 Subject: [PATCH 01/19] [SPARK-48187][INFRA] Run `docs` only in PR builders and `build_non_ansi` Daily CI ### What changes were proposed in this pull request? This PR aims to run `docs` (Documentation Generation) step only in PR builders and `build_non_ansi` Daily CI. To do that, this PR spins off `documentation generation` tasks from `lint` job. ### Why are the changes needed? Currently, Apache Spark CI is running `Documentation Generation` always inside `lint` job. We can take advantage PR Builder and one of Daily CIs. - https://infra.apache.org/github-actions-policy.html ### Does this PR introduce _any_ user-facing change? No because this is an infra update. ### How was this patch tested? Pass the CIs and manual review because PR builders will not be affected by this. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46463 from dongjoon-hyun/SPARK-48187. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 94 ++++++++++++++++++++++++++-- .github/workflows/build_non_ansi.yml | 1 + 2 files changed, 90 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 00ba16265dce4..bb9f2f9a96030 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -85,6 +85,7 @@ jobs: sparkr=`./dev/is-changed.py -m sparkr` buf=true ui=true + docs=true else pandas=false yarn=false @@ -92,6 +93,7 @@ jobs: sparkr=false buf=false ui=false + docs=false fi build=`./dev/is-changed.py -m "core,unsafe,kvstore,avro,utils,network-common,network-shuffle,repl,launcher,examples,sketch,variant,api,catalyst,hive-thriftserver,mllib-local,mllib,graphx,streaming,sql-kafka-0-10,streaming-kafka-0-10,streaming-kinesis-asl,kubernetes,hadoop-cloud,spark-ganglia-lgpl,protobuf,yarn,connect,sql,hive"` precondition=" @@ -103,6 +105,7 @@ jobs: \"tpcds-1g\": \"false\", \"docker-integration-tests\": \"false\", \"lint\" : \"true\", + \"docs\" : \"$docs\", \"yarn\" : \"$yarn\", \"k8s-integration-tests\" : \"$kubernetes\", \"buf\" : \"$buf\", @@ -621,12 +624,12 @@ jobs: - name: Python CodeGen check run: ./dev/connect-check-protos.py - # Static analysis, and documentation build + # Static analysis lint: needs: [precondition, infra-image] # always run if lint == 'true', even infra-image is skip (such as non-master job) if: (!cancelled()) && fromJson(needs.precondition.outputs.required).lint == 'true' - name: Linters, licenses, dependencies and documentation generation + name: Linters, licenses, and dependencies runs-on: ubuntu-latest timeout-minutes: 180 env: @@ -764,7 +767,90 @@ jobs: Rscript -e "devtools::install_version('lintr', version='2.0.1', repos='https://cloud.r-project.org')" - name: Install R linter dependencies and SparkR run: ./R/install-dev.sh - # Should delete this section after SPARK 3.5 EOL. + - name: R linter + run: ./dev/lint-r + + # Documentation build + docs: + needs: [precondition, infra-image] + # always run if lint == 'true', even infra-image is skip (such as non-master job) + if: (!cancelled()) && fromJson(needs.precondition.outputs.required).docs == 'true' + name: Documentation generation + runs-on: ubuntu-latest + timeout-minutes: 180 + env: + LC_ALL: C.UTF-8 + LANG: C.UTF-8 + NOLINT_ON_COMPILE: false + PYSPARK_DRIVER_PYTHON: python3.9 + PYSPARK_PYTHON: python3.9 + GITHUB_PREV_SHA: ${{ github.event.before }} + container: + image: ${{ needs.precondition.outputs.image_url }} + steps: + - name: Checkout Spark repository + uses: actions/checkout@v4 + with: + fetch-depth: 0 + repository: apache/spark + ref: ${{ inputs.branch }} + - name: Add GITHUB_WORKSPACE to git trust safe.directory + run: | + git config --global --add safe.directory ${GITHUB_WORKSPACE} + - name: Sync the current branch with the latest in Apache Spark + if: github.repository != 'apache/spark' + run: | + echo "APACHE_SPARK_REF=$(git rev-parse HEAD)" >> $GITHUB_ENV + git fetch https://github.com/$GITHUB_REPOSITORY.git ${GITHUB_REF#refs/heads/} + git -c user.name='Apache Spark Test Account' -c user.email='sparktestacc@gmail.com' merge --no-commit --progress --squash FETCH_HEAD + git -c user.name='Apache Spark Test Account' -c user.email='sparktestacc@gmail.com' commit -m "Merged commit" --allow-empty + # Cache local repositories. Note that GitHub Actions cache has a 10G limit. + - name: Cache SBT and Maven + uses: actions/cache@v4 + with: + path: | + build/apache-maven-* + build/*.jar + ~/.sbt + key: build-${{ hashFiles('**/pom.xml', 'project/build.properties', 'build/mvn', 'build/sbt', 'build/sbt-launch-lib.bash', 'build/spark-build-info') }} + restore-keys: | + build- + - name: Cache Coursier local repository + uses: actions/cache@v4 + with: + path: ~/.cache/coursier + key: docs-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} + restore-keys: | + docs-coursier- + - name: Cache Maven local repository + uses: actions/cache@v4 + with: + path: ~/.m2/repository + key: docs-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + docs-maven- + - name: Free up disk space + run: | + if [ -f ./dev/free_disk_space_container ]; then + ./dev/free_disk_space_container + fi + - name: Install Java ${{ inputs.java }} + uses: actions/setup-java@v4 + with: + distribution: zulu + java-version: ${{ inputs.java }} + - name: Install Python dependencies for python linter and documentation generation + if: inputs.branch != 'branch-3.4' && inputs.branch != 'branch-3.5' + run: | + # Should unpin 'sphinxcontrib-*' after upgrading sphinx>5 + # See 'ipython_genutils' in SPARK-38517 + # See 'docutils<0.18.0' in SPARK-39421 + 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' '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 list - name: Install dependencies for documentation generation for branch-3.4, branch-3.5 if: inputs.branch == 'branch-3.4' || inputs.branch == 'branch-3.5' run: | @@ -785,8 +871,6 @@ jobs: gem install bundler -v 2.4.22 cd docs bundle install - - name: R linter - run: ./dev/lint-r - name: Run documentation build run: | # We need this link because the jekyll build calls `python`. diff --git a/.github/workflows/build_non_ansi.yml b/.github/workflows/build_non_ansi.yml index 9026276903206..30ead890728c2 100644 --- a/.github/workflows/build_non_ansi.yml +++ b/.github/workflows/build_non_ansi.yml @@ -41,6 +41,7 @@ jobs: jobs: >- { "build": "true", + "docs": "true", "pyspark": "true", "sparkr": "true", "tpcds-1g": "true", From f693abc8de949b1fd5f77b9e74037b0cc2298aef Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 8 May 2024 17:13:11 +0900 Subject: [PATCH 02/19] [SPARK-48192][INFRA] Enable TPC-DS tests in forked repository ### What changes were proposed in this pull request? This PR is a sort of a followup of https://github.com/apache/spark/pull/46361. It proposes to run TPC-DS and Docker integration tests in PRs (that does not consume ASF resources). ### Why are the changes needed? TPC-DS and Docker integration stuff at least have to be tested in the PR if the PR touches the codes related to that. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Manually ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46470 from HyukjinKwon/SPARK-48192. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index bb9f2f9a96030..4a11823aee604 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -83,6 +83,8 @@ jobs: yarn=`./dev/is-changed.py -m yarn` kubernetes=`./dev/is-changed.py -m kubernetes` sparkr=`./dev/is-changed.py -m sparkr` + tpcds=`./dev/is-changed.py -m sql` + docker=`./dev/is-changed.py -m docker-integration-tests` buf=true ui=true docs=true @@ -91,6 +93,8 @@ jobs: yarn=false kubernetes=false sparkr=false + tpcds=false + docker=false buf=false ui=false docs=false @@ -102,8 +106,8 @@ jobs: \"pyspark\": \"$pyspark\", \"pyspark-pandas\": \"$pandas\", \"sparkr\": \"$sparkr\", - \"tpcds-1g\": \"false\", - \"docker-integration-tests\": \"false\", + \"tpcds-1g\": \"$tpcds\", + \"docker-integration-tests\": \"$docker\", \"lint\" : \"true\", \"docs\" : \"$docs\", \"yarn\" : \"$yarn\", From f5401bab23c073410e2b2760baf6007109708d4c Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 8 May 2024 17:15:32 +0900 Subject: [PATCH 03/19] [MINOR][INFRA] Rename builds to have consistent names ### What changes were proposed in this pull request? Minor renaming to match with other names. ### Why are the changes needed? For consistency. ### 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 #46467 from HyukjinKwon/minor-rename. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_non_ansi.yml | 2 +- .github/workflows/cancel_duplicate_workflow_runs.yml | 2 +- .github/workflows/publish_snapshot.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build_non_ansi.yml b/.github/workflows/build_non_ansi.yml index 30ead890728c2..4ac2a589f4f81 100644 --- a/.github/workflows/build_non_ansi.yml +++ b/.github/workflows/build_non_ansi.yml @@ -17,7 +17,7 @@ # under the License. # -name: "Build / NON-ANSI (master, Hadoop 3, JDK 17, Scala 2.13)" +name: "Build / Non-ANSI (master, Hadoop 3, JDK 17, Scala 2.13)" on: schedule: diff --git a/.github/workflows/cancel_duplicate_workflow_runs.yml b/.github/workflows/cancel_duplicate_workflow_runs.yml index d41ca31190d94..697a3fb277d33 100644 --- a/.github/workflows/cancel_duplicate_workflow_runs.yml +++ b/.github/workflows/cancel_duplicate_workflow_runs.yml @@ -17,7 +17,7 @@ # under the License. # -name: Cancelling Duplicates +name: Cancelling duplicates on: workflow_run: workflows: diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index 006ccf239e6fe..1b5bd0ba61288 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -17,7 +17,7 @@ # under the License. # -name: Publish Snapshot +name: Publish snapshot on: schedule: From fe8b18b776f52835090fbbc0cc09d465b15f58ce Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 8 May 2024 18:06:35 +0800 Subject: [PATCH 04/19] [SPARK-48185][SQL] Fix 'symbolic reference class is not accessible: class sun.util.calendar.ZoneInfo' ### What changes were proposed in this pull request? I met the error below while debugging UTs because of loading `sun.util.calendar.ZoneInfo` eagerly. This PR makes the relevant variables lazy. ```log Caused by: java.lang.IllegalAccessException: symbolic reference class is not accessible: class sun.util.calendar.ZoneInfo, from interface org.apache.spark.sql.catalyst.util.SparkDateTimeUtils (unnamed module 65d6b83b) at java.base/java.lang.invoke.MemberName.makeAccessException(MemberName.java:955) at java.base/java.lang.invoke.MethodHandles$Lookup.checkSymbolicClass(MethodHandles.java:3686) at java.base/java.lang.invoke.MethodHandles$Lookup.resolveOrFail(MethodHandles.java:3646) at java.base/java.lang.invoke.MethodHandles$Lookup.findVirtual(MethodHandles.java:2680) at org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.$init$(SparkDateTimeUtils.scala:206) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.(DateTimeUtils.scala:41) ... 82 more ``` ### Why are the changes needed? sun.util.calendar.ZoneInfo is inaccessible in some scenarios. ### Does this PR introduce _any_ user-facing change? Yes, such errors might be delayed from backend-scheduling to job-scheduling ### How was this patch tested? I tested with idea and UT debugging locally ### Was this patch authored or co-authored using generative AI tooling? no Closes #46457 from yaooqinn/SPARK-48185. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala index 8db8c3cd39d74..0447d813e26a5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/SparkDateTimeUtils.scala @@ -198,7 +198,7 @@ trait SparkDateTimeUtils { } private val zoneInfoClassName = "sun.util.calendar.ZoneInfo" - private val getOffsetsByWallHandle = { + private lazy val getOffsetsByWallHandle = { val lookup = MethodHandles.lookup() val classType = SparkClassUtils.classForName(zoneInfoClassName) val methodName = "getOffsetsByWall" From fe3ef20d6418c4ed8965b2d61bf1d32b551e7b53 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 8 May 2024 19:14:46 +0900 Subject: [PATCH 05/19] [SPARK-48193][INFRA] Make `maven-deploy-plugin` retry 3 times ### What changes were proposed in this pull request? The pr aims to make maven plugin `maven-deploy-plugin` retry `3` times. ### Why are the changes needed? I found that our `the daily scheduled publish snapshot` workflow of GA often failed. https://github.com/apache/spark/actions/workflows/publish_snapshot.yml image I tried to make it as successful as possible by changing the time of retries from `1`(default) to `3`. https://maven.apache.org/plugins/maven-deploy-plugin/deploy-mojo.html#retryFailedDeploymentCount https://maven.apache.org/plugins/maven-deploy-plugin/examples/deploy-network-issues.html#configuring-multiple-tries ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Keep observing `the daily scheduled publish snapshot` workflow of GA. https://github.com/apache/spark/actions/workflows/publish_snapshot.yml ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46471 from panbingkun/SPARK-48193. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pom.xml b/pom.xml index f6f11d94cce32..c72482fd6a41f 100644 --- a/pom.xml +++ b/pom.xml @@ -3384,6 +3384,9 @@ org.apache.maven.plugins maven-deploy-plugin 3.1.2 + + 3 + org.apache.maven.plugins From 1b966d2eb329eed45b258d2134aacc0ea62d75dd Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 8 May 2024 19:18:06 +0900 Subject: [PATCH 06/19] [SPARK-47965][SQL][FOLLOW-UP] Uses `null` as its default value for `OptionalConfigEntry` ### What changes were proposed in this pull request? This PR partially reverts https://github.com/apache/spark/pull/46197 because of the behaviour change below: ```python >>> spark.conf.get("spark.sql.optimizer.excludedRules") '' ``` ### Why are the changes needed? To avoid behaviour change. ### Does this PR introduce _any_ user-facing change? No, the main change has not been released out yet. ### How was this patch tested? Manually as described above. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46472 from HyukjinKwon/SPARK-47965-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/internal/config/ConfigEntry.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index c07f2528ee709..a295ef06a6376 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -227,7 +227,7 @@ private[spark] class OptionalConfigEntry[T]( prependSeparator, alternatives, s => Some(rawValueConverter(s)), - v => v.map(rawStringConverter).getOrElse(ConfigEntry.UNDEFINED), + v => v.map(rawStringConverter).orNull, doc, isPublic, version From bd896cac168aa5793413058ca706c73705edbf96 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 8 May 2024 19:28:45 +0900 Subject: [PATCH 07/19] Revert "[SPARK-48163][CONNECT][TESTS] Disable `SparkConnectServiceSuite.SPARK-43923: commands send events - get_resources_command`" This reverts commit 56fe185c78a249cf88b1d7e5d1e67444e1b224db. --- .../spark/sql/connect/planner/SparkConnectServiceSuite.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala index 59d9750c0fbf4..af18fca9dd216 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala @@ -418,14 +418,11 @@ class SparkConnectServiceSuite .setInput( proto.Relation.newBuilder().setSql(proto.SQL.newBuilder().setQuery("select 1")))), None), - // TODO(SPARK-48164) Reenable `commands send events - get_resources_command` - /* ( proto.Command .newBuilder() .setGetResourcesCommand(proto.GetResourcesCommand.newBuilder()), None), - */ ( proto.Command .newBuilder() From d7f69e7003a3c7e7ad22a39e6aaacd183d26d326 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 8 May 2024 18:48:21 +0800 Subject: [PATCH 08/19] [SPARK-48190][PYTHON][PS][TESTS] Introduce a helper function to drop metadata ### What changes were proposed in this pull request? Introduce a helper function to drop metadata ### Why are the changes needed? existing helper function `remove_metadata` in PS doesn't support nested types, so cannot be reused in other places ### 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 #46466 from zhengruifeng/py_drop_meta. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/pandas/internal.py | 17 +++-------------- .../sql/tests/connect/test_connect_function.py | 11 +++++++++-- python/pyspark/sql/types.py | 13 +++++++++++++ 3 files changed, 25 insertions(+), 16 deletions(-) diff --git a/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py index 767ec9a57f9b5..8ab8d79d56868 100644 --- a/python/pyspark/pandas/internal.py +++ b/python/pyspark/pandas/internal.py @@ -33,6 +33,7 @@ Window, ) from pyspark.sql.types import ( # noqa: F401 + _drop_metadata, BooleanType, DataType, LongType, @@ -761,14 +762,8 @@ def __init__( # in a few tests when using Spark Connect. However, the function works properly. # Therefore, we temporarily perform Spark Connect tests by excluding metadata # until the issue is resolved. - def remove_metadata(struct_field: StructField) -> StructField: - new_struct_field = StructField( - struct_field.name, struct_field.dataType, struct_field.nullable - ) - return new_struct_field - assert all( - remove_metadata(index_field.struct_field) == remove_metadata(struct_field) + _drop_metadata(index_field.struct_field) == _drop_metadata(struct_field) for index_field, struct_field in zip(index_fields, struct_fields) ), (index_fields, struct_fields) else: @@ -795,14 +790,8 @@ def remove_metadata(struct_field: StructField) -> StructField: # in a few tests when using Spark Connect. However, the function works properly. # Therefore, we temporarily perform Spark Connect tests by excluding metadata # until the issue is resolved. - def remove_metadata(struct_field: StructField) -> StructField: - new_struct_field = StructField( - struct_field.name, struct_field.dataType, struct_field.nullable - ) - return new_struct_field - assert all( - remove_metadata(data_field.struct_field) == remove_metadata(struct_field) + _drop_metadata(data_field.struct_field) == _drop_metadata(struct_field) for data_field, struct_field in zip(data_fields, struct_fields) ), (data_fields, struct_fields) else: diff --git a/python/pyspark/sql/tests/connect/test_connect_function.py b/python/pyspark/sql/tests/connect/test_connect_function.py index 9d4db8cf7d15d..0f0abfd4b8567 100644 --- a/python/pyspark/sql/tests/connect/test_connect_function.py +++ b/python/pyspark/sql/tests/connect/test_connect_function.py @@ -21,7 +21,14 @@ from pyspark.util import is_remote_only from pyspark.errors import PySparkTypeError, PySparkValueError from pyspark.sql import SparkSession as PySparkSession -from pyspark.sql.types import StringType, StructType, StructField, ArrayType, IntegerType +from pyspark.sql.types import ( + _drop_metadata, + StringType, + StructType, + StructField, + ArrayType, + IntegerType, +) from pyspark.testing import assertDataFrameEqual from pyspark.testing.pandasutils import PandasOnSparkTestUtils from pyspark.testing.connectutils import ReusedConnectTestCase, should_test_connect @@ -1668,7 +1675,7 @@ def test_nested_lambda_function(self): ) # TODO: 'cdf.schema' has an extra metadata '{'__autoGeneratedAlias': 'true'}' - # self.assertEqual(cdf.schema, sdf.schema) + self.assertEqual(_drop_metadata(cdf.schema), _drop_metadata(sdf.schema)) self.assertEqual(cdf.collect(), sdf.collect()) def test_csv_functions(self): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 48aa3e8e4faba..41be12620fd56 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1569,6 +1569,19 @@ def toJson(self, zone_id: str = "UTC") -> str: _INTERVAL_YEARMONTH = re.compile(r"interval (year|month)( to (year|month))?") +def _drop_metadata(d: Union[DataType, StructField]) -> Union[DataType, StructField]: + assert isinstance(d, (DataType, StructField)) + if isinstance(d, StructField): + return StructField(d.name, _drop_metadata(d.dataType), d.nullable, None) + elif isinstance(d, StructType): + return StructType([cast(StructField, _drop_metadata(f)) for f in d.fields]) + elif isinstance(d, ArrayType): + return ArrayType(_drop_metadata(d.elementType), d.containsNull) + elif isinstance(d, MapType): + return MapType(_drop_metadata(d.keyType), _drop_metadata(d.valueType), d.valueContainsNull) + return d + + def _parse_datatype_string(s: str) -> DataType: """ Parses the given data type string to a :class:`DataType`. The data type string format equals From 003823b39d3504a2a2cffaabbcab1dcf9429fa81 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Wed, 8 May 2024 20:09:22 +0800 Subject: [PATCH 09/19] [SPARK-48191][SQL] Support UTF-32 for string encode and decode ### What changes were proposed in this pull request? Enable support of UTF-32 ### Why are the changes needed? It already works, so we just need to enable it ### Does this PR introduce _any_ user-facing change? Yes, `decode(..., 'UTF-32')` and `encode(..., 'UTF-32')` will start working ### How was this patch tested? Manually checked in the spark shell ### Was this patch authored or co-authored using generative AI tooling? No Closes #46469 from vladimirg-db/vladimirg-db/support-utf-32-for-string-decode. Authored-by: Vladimir Golubev Signed-off-by: Kent Yao --- docs/sql-migration-guide.md | 2 +- .../sql/catalyst/expressions/stringExpressions.scala | 10 +++++----- .../catalyst/expressions/StringExpressionsSuite.scala | 2 ++ .../analyzer-results/ansi/string-functions.sql.out | 7 +++++++ .../analyzer-results/string-functions.sql.out | 7 +++++++ .../resources/sql-tests/inputs/string-functions.sql | 1 + .../sql-tests/results/ansi/string-functions.sql.out | 8 ++++++++ .../sql-tests/results/string-functions.sql.out | 8 ++++++++ 8 files changed, 39 insertions(+), 6 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index fa49d6402b180..bd6604cb69c0f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -32,7 +32,7 @@ license: | - 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. -- 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'. 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 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 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` 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 c2ea17de19533..0bdd7930b0bf9 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 @@ -2646,7 +2646,7 @@ object Decode { arguments = """ Arguments: * bin - a binary expression to decode - * charset - one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16' to decode `bin` into a STRING. It is case insensitive. + * charset - one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32' to decode `bin` into a STRING. It is case insensitive. """, examples = """ Examples: @@ -2690,7 +2690,7 @@ case class Decode(params: Seq[Expression], replacement: Expression) arguments = """ Arguments: * bin - a binary expression to decode - * charset - one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16' to decode `bin` into a STRING. It is case insensitive. + * charset - one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32' to decode `bin` into a STRING. It is case insensitive. """, since = "1.5.0", group = "string_funcs") @@ -2707,7 +2707,7 @@ case class StringDecode(bin: Expression, charset: Expression, legacyCharsets: Bo override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType, StringTypeAnyCollation) private val supportedCharsets = Set( - "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16") + "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16", "UTF-32") protected override def nullSafeEval(input1: Any, input2: Any): Any = { val fromCharset = input2.asInstanceOf[UTF8String].toString @@ -2762,7 +2762,7 @@ object StringDecode { arguments = """ Arguments: * str - a string expression - * charset - one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16' to encode `str` into a BINARY. It is case insensitive. + * charset - one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', 'UTF-32' to encode `str` into a BINARY. It is case insensitive. """, examples = """ Examples: @@ -2785,7 +2785,7 @@ case class Encode(str: Expression, charset: Expression, legacyCharsets: Boolean) Seq(StringTypeAnyCollation, StringTypeAnyCollation) private val supportedCharsets = Set( - "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16") + "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16", "UTF-32") protected override def nullSafeEval(input1: Any, input2: Any): Any = { val toCharset = input2.asInstanceOf[UTF8String].toString diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 0fcceef392389..51de44d8dfd98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -489,6 +489,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // non ascii characters are not allowed in the code, so we disable the scalastyle here. checkEvaluation( StringDecode(Encode(Literal("大千世界"), Literal("UTF-16LE")), Literal("UTF-16LE")), "大千世界") + checkEvaluation( + StringDecode(Encode(Literal("大千世界"), Literal("UTF-32")), Literal("UTF-32")), "大千世界") checkEvaluation( StringDecode(Encode(a, Literal("utf-8")), Literal("utf-8")), "大千世界", create_row("大千世界")) checkEvaluation( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out index 7ffd3cbd8bac6..c36dec0b105d7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out @@ -750,6 +750,13 @@ Project [decode(encode(abc, utf-8, false), utf-8) AS decode(encode(abc, utf-8), +- OneRowRelation +-- !query +select decode(encode('大千世界', 'utf-32'), 'utf-32') +-- !query analysis +Project [decode(encode(大千世界, utf-32, false), utf-32) AS decode(encode(大千世界, utf-32), utf-32)#x] ++- OneRowRelation + + -- !query select decode(1, 1, 'Southlake') -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out index 7ffd3cbd8bac6..c36dec0b105d7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/string-functions.sql.out @@ -750,6 +750,13 @@ Project [decode(encode(abc, utf-8, false), utf-8) AS decode(encode(abc, utf-8), +- OneRowRelation +-- !query +select decode(encode('大千世界', 'utf-32'), 'utf-32') +-- !query analysis +Project [decode(encode(大千世界, utf-32, false), utf-32) AS decode(encode(大千世界, utf-32), utf-32)#x] ++- OneRowRelation + + -- !query select decode(1, 1, 'Southlake') -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql index 64ea6e655d0b5..733720a7e21b2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql @@ -131,6 +131,7 @@ select encode(scol, ecol) from values('hello', 'Windows-xxx') as t(scol, ecol); select decode(); select decode(encode('abc', 'utf-8')); select decode(encode('abc', 'utf-8'), 'utf-8'); +select decode(encode('大千世界', 'utf-32'), 'utf-32'); select decode(1, 1, 'Southlake'); select decode(2, 1, 'Southlake'); select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic'); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index 8096cef266ec4..09d4f8892fa48 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -961,6 +961,14 @@ struct abc +-- !query +select decode(encode('大千世界', 'utf-32'), 'utf-32') +-- !query schema +struct +-- !query output +大千世界 + + -- !query select decode(1, 1, 'Southlake') -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 91ad830dd3d7a..506524840f107 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -893,6 +893,14 @@ struct abc +-- !query +select decode(encode('大千世界', 'utf-32'), 'utf-32') +-- !query schema +struct +-- !query output +大千世界 + + -- !query select decode(1, 1, 'Southlake') -- !query schema From 8950add773e63a910900f796950a6a58e40a8577 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 May 2024 20:11:24 +0800 Subject: [PATCH 10/19] [SPARK-48188][SQL] Consistently use normalized plan for cache ### What changes were proposed in this pull request? We must consistently use normalized plans for cache filling and lookup, or inconsistency will lead to cache misses. To guarantee this, this PR makes `CacheManager` the central place to do plan normalization, so that callers don't need to care about it. Now most APIs in `CacheManager` take either `Dataset` or `LogicalPlan`. For `Dataset`, we get the normalized plan directly. For `LogicalPlan`, we normalize it before further use. The caller side should pass `Dataset` when invoking `CacheManager`, if it already creates `Dataset`. This is to reduce the impact, as extra creation of `Dataset` may have perf issues or introduce unexpected analysis exception. ### Why are the changes needed? Avoid unnecessary cache misses for users who add custom normalization rules ### Does this PR introduce _any_ user-facing change? No, perf only ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #46465 from cloud-fan/cache. Authored-by: Wenchen Fan Signed-off-by: Kent Yao --- .../scala/org/apache/spark/sql/Dataset.scala | 3 +- .../spark/sql/execution/CacheManager.scala | 160 +++++++++++------- .../spark/sql/execution/QueryExecution.scala | 37 ++-- .../command/AnalyzeColumnCommand.scala | 4 +- .../sql/execution/command/CommandUtils.scala | 2 +- .../datasources/v2/CacheTableExec.scala | 30 ++-- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../spark/sql/internal/CatalogImpl.scala | 5 +- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../apache/spark/sql/test/SQLTestUtils.scala | 3 +- .../spark/sql/hive/CachedTableSuite.scala | 9 +- 11 files changed, 150 insertions(+), 107 deletions(-) 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 18c9704afdf83..3e843e64ebbf6 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 @@ -3904,8 +3904,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def unpersist(blocking: Boolean): this.type = { - sparkSession.sharedState.cacheManager.uncacheQuery( - sparkSession, logicalPlan, cascade = false, blocking) + sparkSession.sharedState.cacheManager.uncacheQuery(this, cascade = false, blocking) this } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index ae99873a9f774..b96f257e6b5b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint -import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint, View} import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -38,7 +38,10 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK /** Holds a cached logical plan and its data */ -case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) { +case class CachedData( + // A normalized resolved plan (See QueryExecution#normalized). + plan: LogicalPlan, + cachedRepresentation: InMemoryRelation) { override def toString: String = s""" |CachedData( @@ -53,7 +56,9 @@ case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) * InMemoryRelation. This relation is automatically substituted query plans that return the * `sameResult` as the originally cached query. * - * Internal to Spark SQL. + * Internal to Spark SQL. All its public APIs take analyzed plans and will normalize them before + * further usage, or take [[Dataset]] and get its normalized plan. See `QueryExecution.normalize` + * for more details about plan normalization. */ class CacheManager extends Logging with AdaptiveSparkPlanHelper { @@ -77,41 +82,43 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { cachedData.isEmpty } + // Test-only + def cacheQuery(query: Dataset[_]): Unit = { + cacheQuery(query, tableName = None, storageLevel = MEMORY_AND_DISK) + } + /** * Caches the data produced by the logical representation of the given [[Dataset]]. - * Unlike `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because - * recomputing the in-memory columnar representation of the underlying table is expensive. */ def cacheQuery( query: Dataset[_], - tableName: Option[String] = None, - storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = { - cacheQuery(query.sparkSession, query.queryExecution.normalized, tableName, storageLevel) + tableName: Option[String], + storageLevel: StorageLevel): Unit = { + cacheQueryInternal(query.sparkSession, query.queryExecution.normalized, tableName, storageLevel) } /** - * Caches the data produced by the given [[LogicalPlan]]. - * Unlike `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because - * recomputing the in-memory columnar representation of the underlying table is expensive. + * Caches the data produced by the given [[LogicalPlan]]. The given plan will be normalized + * before being used further. */ def cacheQuery( spark: SparkSession, planToCache: LogicalPlan, - tableName: Option[String]): Unit = { - cacheQuery(spark, planToCache, tableName, MEMORY_AND_DISK) + tableName: Option[String], + storageLevel: StorageLevel): Unit = { + val normalized = QueryExecution.normalize(spark, planToCache) + cacheQueryInternal(spark, normalized, tableName, storageLevel) } - /** - * Caches the data produced by the given [[LogicalPlan]]. - */ - def cacheQuery( + // The `planToCache` should have been normalized. + private def cacheQueryInternal( spark: SparkSession, planToCache: LogicalPlan, tableName: Option[String], storageLevel: StorageLevel): Unit = { if (storageLevel == StorageLevel.NONE) { // Do nothing for StorageLevel.NONE since it will not actually cache any data. - } else if (lookupCachedData(planToCache).nonEmpty) { + } else if (lookupCachedDataInternal(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") } else { val sessionWithConfigsOff = getOrCloneSessionWithConfigsOff(spark) @@ -124,7 +131,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { } this.synchronized { - if (lookupCachedData(planToCache).nonEmpty) { + if (lookupCachedDataInternal(planToCache).nonEmpty) { logWarning("Data has already been cached.") } else { val cd = CachedData(planToCache, inMemoryRelation) @@ -138,38 +145,64 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { /** * Un-cache the given plan or all the cache entries that refer to the given plan. - * @param query The [[Dataset]] to be un-cached. - * @param cascade If true, un-cache all the cache entries that refer to the given - * [[Dataset]]; otherwise un-cache the given [[Dataset]] only. + * + * @param query The [[Dataset]] to be un-cached. + * @param cascade If true, un-cache all the cache entries that refer to the given + * [[Dataset]]; otherwise un-cache the given [[Dataset]] only. + * @param blocking Whether to block until all blocks are deleted. */ + def uncacheQuery( + query: Dataset[_], + cascade: Boolean, + blocking: Boolean): Unit = { + uncacheQueryInternal(query.sparkSession, query.queryExecution.normalized, cascade, blocking) + } + + // An overload to provide default value for the `blocking` parameter. def uncacheQuery( query: Dataset[_], cascade: Boolean): Unit = { - uncacheQuery(query.sparkSession, query.queryExecution.normalized, cascade) + uncacheQuery(query, cascade, blocking = false) } /** * Un-cache the given plan or all the cache entries that refer to the given plan. - * @param spark The Spark session. - * @param plan The plan to be un-cached. - * @param cascade If true, un-cache all the cache entries that refer to the given - * plan; otherwise un-cache the given plan only. - * @param blocking Whether to block until all blocks are deleted. + * + * @param spark The Spark session. + * @param plan The plan to be un-cached. + * @param cascade If true, un-cache all the cache entries that refer to the given + * plan; otherwise un-cache the given plan only. + * @param blocking Whether to block until all blocks are deleted. */ def uncacheQuery( spark: SparkSession, plan: LogicalPlan, cascade: Boolean, - blocking: Boolean = false): Unit = { - uncacheQuery(spark, _.sameResult(plan), cascade, blocking) + blocking: Boolean): Unit = { + val normalized = QueryExecution.normalize(spark, plan) + uncacheQueryInternal(spark, normalized, cascade, blocking) + } + + // An overload to provide default value for the `blocking` parameter. + def uncacheQuery( + spark: SparkSession, + plan: LogicalPlan, + cascade: Boolean): Unit = { + uncacheQuery(spark, plan, cascade, blocking = false) + } + + // The `plan` should have been normalized. + private def uncacheQueryInternal( + spark: SparkSession, + plan: LogicalPlan, + cascade: Boolean, + blocking: Boolean): Unit = { + uncacheByCondition(spark, _.sameResult(plan), cascade, blocking) } def uncacheTableOrView(spark: SparkSession, name: Seq[String], cascade: Boolean): Unit = { - uncacheQuery( - spark, - isMatchedTableOrView(_, name, spark.sessionState.conf), - cascade, - blocking = false) + uncacheByCondition( + spark, isMatchedTableOrView(_, name, spark.sessionState.conf), cascade, blocking = false) } private def isMatchedTableOrView(plan: LogicalPlan, name: Seq[String], conf: SQLConf): Boolean = { @@ -178,28 +211,24 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { } plan match { - case SubqueryAlias(ident, LogicalRelation(_, _, Some(catalogTable), _)) => - val v1Ident = catalogTable.identifier - isSameName(ident.qualifier :+ ident.name) && isSameName(v1Ident.nameParts) + case LogicalRelation(_, _, Some(catalogTable), _) => + isSameName(catalogTable.identifier.nameParts) - case SubqueryAlias(ident, DataSourceV2Relation(_, _, Some(catalog), Some(v2Ident), _)) => + case DataSourceV2Relation(_, _, Some(catalog), Some(v2Ident), _) => import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper - isSameName(ident.qualifier :+ ident.name) && - isSameName(v2Ident.toQualifiedNameParts(catalog)) + isSameName(v2Ident.toQualifiedNameParts(catalog)) - case SubqueryAlias(ident, View(catalogTable, _, _)) => - val v1Ident = catalogTable.identifier - isSameName(ident.qualifier :+ ident.name) && isSameName(v1Ident.nameParts) + case View(catalogTable, _, _) => + isSameName(catalogTable.identifier.nameParts) - case SubqueryAlias(ident, HiveTableRelation(catalogTable, _, _, _, _)) => - val v1Ident = catalogTable.identifier - isSameName(ident.qualifier :+ ident.name) && isSameName(v1Ident.nameParts) + case HiveTableRelation(catalogTable, _, _, _, _) => + isSameName(catalogTable.identifier.nameParts) case _ => false } } - def uncacheQuery( + private def uncacheByCondition( spark: SparkSession, isMatchedPlan: LogicalPlan => Boolean, cascade: Boolean, @@ -252,10 +281,12 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { } /** - * Tries to re-cache all the cache entries that refer to the given plan. + * Tries to re-cache all the cache entries that refer to the given plan. The given plan will be + * normalized before being used further. */ def recacheByPlan(spark: SparkSession, plan: LogicalPlan): Unit = { - recacheByCondition(spark, _.plan.exists(_.sameResult(plan))) + val normalized = QueryExecution.normalize(spark, plan) + recacheByCondition(spark, _.plan.exists(_.sameResult(normalized))) } /** @@ -278,7 +309,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { } val recomputedPlan = cd.copy(cachedRepresentation = newCache) this.synchronized { - if (lookupCachedData(recomputedPlan.plan).nonEmpty) { + if (lookupCachedDataInternal(recomputedPlan.plan).nonEmpty) { logWarning("While recaching, data was already added to cache.") } else { cachedData = recomputedPlan +: cachedData @@ -289,13 +320,23 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { } } - /** Optionally returns cached data for the given [[Dataset]] */ + /** + * Optionally returns cached data for the given [[Dataset]] + */ def lookupCachedData(query: Dataset[_]): Option[CachedData] = { - lookupCachedData(query.queryExecution.normalized) + lookupCachedDataInternal(query.queryExecution.normalized) } - /** Optionally returns cached data for the given [[LogicalPlan]]. */ - def lookupCachedData(plan: LogicalPlan): Option[CachedData] = { + /** + * Optionally returns cached data for the given [[LogicalPlan]]. The given plan will be normalized + * before being used further. + */ + def lookupCachedData(session: SparkSession, plan: LogicalPlan): Option[CachedData] = { + val normalized = QueryExecution.normalize(session, plan) + lookupCachedDataInternal(normalized) + } + + private def lookupCachedDataInternal(plan: LogicalPlan): Option[CachedData] = { val result = cachedData.find(cd => plan.sameResult(cd.plan)) if (result.isDefined) { CacheManager.logCacheOperation(log"Dataframe cache hit for input plan:" + @@ -305,13 +346,16 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { result } - /** Replaces segments of the given logical plan with cached versions where possible. */ - def useCachedData(plan: LogicalPlan): LogicalPlan = { + /** + * Replaces segments of the given logical plan with cached versions where possible. The input + * plan must be normalized. + */ + private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = { val newPlan = plan transformDown { case command: IgnoreCachedData => command case currentFragment => - lookupCachedData(currentFragment).map { cached => + lookupCachedDataInternal(currentFragment).map { cached => // After cache lookup, we should still keep the hints from the input plan. val hints = EliminateResolvedHint.extractHintsFromPlan(currentFragment)._2 val cachedPlan = cached.cachedRepresentation.withOutput(currentFragment.output) 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 d04d8dc2cd7fd..357484ca19df2 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 @@ -134,19 +134,7 @@ class QueryExecution( // The plan that has been normalized by custom rules, so that it's more likely to hit cache. lazy val normalized: LogicalPlan = { - val normalizationRules = sparkSession.sessionState.planNormalizationRules - if (normalizationRules.isEmpty) { - commandExecuted - } else { - val planChangeLogger = new PlanChangeLogger[LogicalPlan]() - val normalized = normalizationRules.foldLeft(commandExecuted) { (p, rule) => - val result = rule.apply(p) - planChangeLogger.logRule(rule.ruleName, p, result) - result - } - planChangeLogger.logBatch("Plan Normalization", commandExecuted, normalized) - normalized - } + QueryExecution.normalize(sparkSession, commandExecuted, Some(tracker)) } lazy val withCachedData: LogicalPlan = sparkSession.withActive { @@ -613,4 +601,27 @@ object QueryExecution { case e: Throwable => throw toInternalError(msg, e) } } + + def normalize( + session: SparkSession, + plan: LogicalPlan, + tracker: Option[QueryPlanningTracker] = None): LogicalPlan = { + val normalizationRules = session.sessionState.planNormalizationRules + if (normalizationRules.isEmpty) { + plan + } else { + val planChangeLogger = new PlanChangeLogger[LogicalPlan]() + val normalized = normalizationRules.foldLeft(plan) { (p, rule) => + val startTime = System.nanoTime() + val result = rule.apply(p) + val runTime = System.nanoTime() - startTime + val effective = !result.fastEquals(p) + tracker.foreach(_.recordRuleInvocation(rule.ruleName, runTime, effective)) + planChangeLogger.logRule(rule.ruleName, p, result) + result + } + planChangeLogger.logBatch("Plan Normalization", plan, normalized) + normalized + } + } } 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 299f41eb55e17..7b0ce3e59263f 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 @@ -61,8 +61,8 @@ case class AnalyzeColumnCommand( private def analyzeColumnInCachedData(plan: LogicalPlan, sparkSession: SparkSession): Boolean = { val cacheManager = sparkSession.sharedState.cacheManager - val planToLookup = sparkSession.sessionState.executePlan(plan).analyzed - cacheManager.lookupCachedData(planToLookup).map { cachedData => + val df = Dataset.ofRows(sparkSession, plan) + cacheManager.lookupCachedData(df).map { cachedData => val columnsToAnalyze = getColumnsToAnalyze( tableIdent, cachedData.cachedRepresentation, columnNames, allColumns) cacheManager.analyzeColumnCacheQuery(sparkSession, cachedData, columnsToAnalyze) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index d7c5df151bf12..7acd1cb0852b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -240,7 +240,7 @@ object CommandUtils extends Logging { // Analyzes a catalog view if the view is cached val table = sparkSession.table(tableIdent.quotedString) val cacheManager = sparkSession.sharedState.cacheManager - if (cacheManager.lookupCachedData(table.logicalPlan).isDefined) { + if (cacheManager.lookupCachedData(table).isDefined) { if (!noScan) { // To collect table stats, materializes an underlying columnar RDD table.count() 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 fc8a40f885450..56c44a1256815 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 @@ -21,9 +21,9 @@ import java.util.Locale import org.apache.spark.internal.LogKeys.OPTIONS import org.apache.spark.internal.MDC -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.LocalTempView +import org.apache.spark.sql.catalyst.analysis.{LocalTempView, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap @@ -34,7 +34,6 @@ import org.apache.spark.storage.StorageLevel trait BaseCacheTableExec extends LeafV2CommandExec { def relationName: String def planToCache: LogicalPlan - def dataFrameForCachedPlan: DataFrame def isLazy: Boolean def options: Map[String, String] @@ -49,15 +48,12 @@ trait BaseCacheTableExec extends LeafV2CommandExec { logWarning(log"Invalid options: ${MDC(OPTIONS, withoutStorageLevel.mkString(", "))}") } - session.sharedState.cacheManager.cacheQuery( - session, - planToCache, - Some(relationName), - storageLevel) + val df = Dataset.ofRows(session, planToCache) + session.sharedState.cacheManager.cacheQuery(df, Some(relationName), storageLevel) if (!isLazy) { // Performs eager caching. - dataFrameForCachedPlan.count() + df.count() } Seq.empty @@ -74,10 +70,6 @@ case class CacheTableExec( override lazy val relationName: String = multipartIdentifier.quoted override lazy val planToCache: LogicalPlan = relation - - override lazy val dataFrameForCachedPlan: DataFrame = { - Dataset.ofRows(session, planToCache) - } } case class CacheTableAsSelectExec( @@ -89,7 +81,10 @@ case class CacheTableAsSelectExec( referredTempFunctions: Seq[String]) extends BaseCacheTableExec { override lazy val relationName: String = tempViewName - override lazy val planToCache: LogicalPlan = { + override def planToCache: LogicalPlan = UnresolvedRelation(Seq(tempViewName)) + + override def run(): Seq[InternalRow] = { + // CACHE TABLE AS TABLE creates a temp view and caches the temp view. CreateViewCommand( name = TableIdentifier(tempViewName), userSpecifiedColumns = Nil, @@ -103,12 +98,7 @@ case class CacheTableAsSelectExec( isAnalyzed = true, referredTempFunctions = referredTempFunctions ).run(session) - - dataFrameForCachedPlan.logicalPlan - } - - override lazy val dataFrameForCachedPlan: DataFrame = { - session.table(tempViewName) + super.run() } } 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 0d926dcd99c4a..7a668b75c3c73 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 @@ -83,7 +83,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat // given table, the cache's storage level is returned. private def invalidateTableCache(r: ResolvedTable)(): Option[StorageLevel] = { val v2Relation = DataSourceV2Relation.create(r.table, Some(r.catalog), Some(r.identifier)) - val cache = session.sharedState.cacheManager.lookupCachedData(v2Relation) + val cache = session.sharedState.cacheManager.lookupCachedData(session, v2Relation) session.sharedState.cacheManager.uncacheQuery(session, v2Relation, cascade = true) if (cache.isDefined) { val cacheLevel = cache.get.cachedRepresentation.cacheBuilder.storageLevel 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 df7c4ab1a0c7d..3e20a23a0a066 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 @@ -734,9 +734,8 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // same way as how a permanent view is handled. This also avoids a potential issue where a // dependent view becomes invalid because of the above while its data is still cached. val viewText = viewDef.desc.viewText - val plan = sparkSession.sessionState.executePlan(viewDef) - sparkSession.sharedState.cacheManager.uncacheQuery( - sparkSession, plan.analyzed, cascade = viewText.isDefined) + val df = Dataset.ofRows(sparkSession, viewDef) + sparkSession.sharedState.cacheManager.uncacheQuery(df, cascade = viewText.isDefined) } catch { case NonFatal(_) => // ignore } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 0ad9ceefc4196..d023fb82185a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -1107,7 +1107,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils assert(queryStats1.map(_._1.name).isEmpty) val cacheManager = spark.sharedState.cacheManager - val cachedData = cacheManager.lookupCachedData(query().logicalPlan) + val cachedData = cacheManager.lookupCachedData(query()) assert(cachedData.isDefined) val queryAttrs = cachedData.get.plan.output assert(queryAttrs.size === 3) 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 7da2bb47038ed..5fbf379644f6d 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 @@ -339,8 +339,7 @@ private[sql] trait SQLTestUtilsBase val tableIdent = spark.sessionState.sqlParser.parseTableIdentifier(tableName) val cascade = !spark.sessionState.catalog.isTempView(tableIdent) spark.sharedState.cacheManager.uncacheQuery( - spark, - spark.table(tableName).logicalPlan, + spark.table(tableName), cascade = cascade, blocking = true) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 89fe10d5c4bd9..d7918f8cbf4f0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -335,9 +335,10 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto options = Map.empty)(sparkSession = spark) val plan = LogicalRelation(relation, tableMeta) - spark.sharedState.cacheManager.cacheQuery(Dataset.ofRows(spark, plan)) + val df = Dataset.ofRows(spark, plan) + spark.sharedState.cacheManager.cacheQuery(df) - assert(spark.sharedState.cacheManager.lookupCachedData(plan).isDefined) + assert(spark.sharedState.cacheManager.lookupCachedData(df).isDefined) val sameCatalog = new CatalogFileIndex(spark, tableMeta, 0) val sameRelation = HadoopFsRelation( @@ -347,9 +348,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto bucketSpec = None, fileFormat = new ParquetFileFormat(), options = Map.empty)(sparkSession = spark) - val samePlan = LogicalRelation(sameRelation, tableMeta) + val samePlanDf = Dataset.ofRows(spark, LogicalRelation(sameRelation, tableMeta)) - assert(spark.sharedState.cacheManager.lookupCachedData(samePlan).isDefined) + assert(spark.sharedState.cacheManager.lookupCachedData(samePlanDf).isDefined) } } From 8d7081639ab47996e357a0a968ca74661795da85 Mon Sep 17 00:00:00 2001 From: Uros Bojanic <157381213+uros-db@users.noreply.github.com> Date: Wed, 8 May 2024 20:57:08 +0800 Subject: [PATCH 11/19] [SPARK-48161][SQL] Add collation support for JSON expressions ### What changes were proposed in this pull request? Introduce collation awareness for JSON expressions: get_json_object, json_tuple, from_json, to_json, json_array_length, json_object_keys. ### Why are the changes needed? Add collation support for JSON expressions in Spark. ### Does this PR introduce _any_ user-facing change? Yes, users should now be able to use collated strings within arguments for JSON functions: get_json_object, json_tuple, from_json, to_json, json_array_length, json_object_keys. ### How was this patch tested? E2e sql tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46462 from uros-db/json-expressions. Authored-by: Uros Bojanic <157381213+uros-db@users.noreply.github.com> Signed-off-by: Wenchen Fan --- .../expressions/complexTypeCreator.scala | 5 +- .../expressions/jsonExpressions.scala | 20 +- .../sql/catalyst/json/JacksonParser.scala | 2 +- .../sql/CollationSQLExpressionsSuite.scala | 198 ++++++++++++++++++ 4 files changed, 213 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index c38b6cea9a0a5..4c0d005340606 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -374,7 +374,8 @@ object CreateStruct { // We should always use the last part of the column name (`c` in the above example) as the // alias name inside CreateNamedStruct. case (u: UnresolvedAttribute, _) => Seq(Literal(u.nameParts.last), u) - case (u @ UnresolvedExtractValue(_, e: Literal), _) if e.dataType == StringType => Seq(e, u) + case (u @ UnresolvedExtractValue(_, e: Literal), _) if e.dataType.isInstanceOf[StringType] => + Seq(e, u) case (a: Alias, _) => Seq(Literal(a.name), a) case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) case (e: NamedExpression, _) => Seq(NamePlaceholder, e) @@ -465,7 +466,7 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression with toSQLId(prettyName), Seq("2n (n > 0)"), children.length ) } else { - val invalidNames = nameExprs.filterNot(e => e.foldable && e.dataType == StringType) + val invalidNames = nameExprs.filterNot(e => e.foldable && e.dataType.isInstanceOf[StringType]) if (invalidNames.nonEmpty) { DataTypeMismatch( errorSubClass = "CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING", 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 35e30ceb45cb5..8258bb389e2da 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 @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, TreePatt import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.types.StringTypeAnyCollation import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{UTF8String, VariantVal} import org.apache.spark.util.Utils @@ -132,8 +133,9 @@ case class GetJsonObject(json: Expression, path: Expression) override def left: Expression = json override def right: Expression = path - override def inputTypes: Seq[DataType] = Seq(StringType, StringType) - override def dataType: DataType = StringType + override def inputTypes: Seq[AbstractDataType] = + Seq(StringTypeAnyCollation, StringTypeAnyCollation) + override def dataType: DataType = SQLConf.get.defaultStringType override def nullable: Boolean = true override def prettyName: String = "get_json_object" @@ -477,7 +479,7 @@ case class JsonTuple(children: Seq[Expression]) @transient private lazy val constantFields: Int = foldableFieldNames.count(_ != null) override def elementSchema: StructType = StructType(fieldExpressions.zipWithIndex.map { - case (_, idx) => StructField(s"c$idx", StringType, nullable = true) + case (_, idx) => StructField(s"c$idx", children.head.dataType, nullable = true) }) override def prettyName: String = "json_tuple" @@ -487,7 +489,7 @@ case class JsonTuple(children: Seq[Expression]) throw QueryCompilationErrors.wrongNumArgsError( toSQLId(prettyName), Seq("> 1"), children.length ) - } else if (children.forall(child => StringType.acceptsType(child.dataType))) { + } else if (children.forall(child => StringTypeAnyCollation.acceptsType(child.dataType))) { TypeCheckResult.TypeCheckSuccess } else { DataTypeMismatch( @@ -722,7 +724,7 @@ case class JsonToStructs( converter(parser.parse(json.asInstanceOf[UTF8String])) } - override def inputTypes: Seq[AbstractDataType] = StringType :: Nil + override def inputTypes: Seq[AbstractDataType] = StringTypeAnyCollation :: Nil override def sql: String = schema match { case _: MapType => "entries" @@ -824,7 +826,7 @@ case class StructsToJson( } } - override def dataType: DataType = StringType + override def dataType: DataType = SQLConf.get.defaultStringType override def checkInputDataTypes(): TypeCheckResult = inputSchema match { case dt @ (_: StructType | _: MapType | _: ArrayType | _: VariantType) => @@ -957,7 +959,7 @@ case class SchemaOfJson( case class LengthOfJsonArray(child: Expression) extends UnaryExpression with CodegenFallback with ExpectsInputTypes { - override def inputTypes: Seq[DataType] = Seq(StringType) + override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeAnyCollation) override def dataType: DataType = IntegerType override def nullable: Boolean = true override def prettyName: String = "json_array_length" @@ -1030,8 +1032,8 @@ case class LengthOfJsonArray(child: Expression) extends UnaryExpression case class JsonObjectKeys(child: Expression) extends UnaryExpression with CodegenFallback with ExpectsInputTypes { - override def inputTypes: Seq[DataType] = Seq(StringType) - override def dataType: DataType = ArrayType(StringType) + override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeAnyCollation) + 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/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 3c42f72fa6b6c..848c20ee36bef 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 @@ -275,7 +275,7 @@ class JacksonParser( } } - case StringType => + case _: StringType => (parser: JsonParser) => parseJsonToken[UTF8String](parser, dataType) { case VALUE_STRING => UTF8String.fromString(parser.getText) 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 4314ff97a3cf3..19f34ec15aa07 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 @@ -460,6 +460,204 @@ class CollationSQLExpressionsSuite }) } + test("Support GetJsonObject json expression with collation") { + case class GetJsonObjectTestCase( + input: String, + path: String, + collationName: String, + result: String + ) + + val testCases = Seq( + GetJsonObjectTestCase("{\"a\":\"b\"}", "$.a", "UTF8_BINARY", "b"), + GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_BINARY_LCASE", "1"), + GetJsonObjectTestCase("{\"x\":true}", "$.x", "UNICODE", "true"), + GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI", "1") + ) + + // Supported collations + testCases.foreach(t => { + val query = + s""" + |SELECT get_json_object('${t.input}', '${t.path}') + |""".stripMargin + // Result & data type + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { + val testQuery = sql(query) + checkAnswer(testQuery, Row(t.result)) + val dataType = StringType(t.collationName) + assert(testQuery.schema.fields.head.dataType.sameType(dataType)) + } + }) + } + + test("Support JsonTuple json expression with collation") { + case class JsonTupleTestCase( + input: String, + names: String, + collationName: String, + result: Row + ) + + val testCases = Seq( + JsonTupleTestCase("{\"a\":1, \"b\":2}", "'a', 'b'", "UTF8_BINARY", + Row("1", "2")), + JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_BINARY_LCASE", + Row("3", "4")), + JsonTupleTestCase("{\"x\":true, \"y\":false}", "'x', 'y'", "UNICODE", + Row("true", "false")), + JsonTupleTestCase("{\"X\":null, \"Y\":null}", "'X', 'Y'", "UNICODE_CI", + Row(null, null)) + ) + + // Supported collations + testCases.foreach(t => { + val query = + s""" + |SELECT json_tuple('${t.input}', ${t.names}) + |""".stripMargin + // Result & data type + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { + val testQuery = sql(query) + checkAnswer(testQuery, t.result) + val dataType = StringType(t.collationName) + assert(testQuery.schema.fields.head.dataType.sameType(dataType)) + } + }) + } + + test("Support JsonToStructs json expression with collation") { + case class JsonToStructsTestCase( + input: String, + schema: String, + collationName: String, + result: Row + ) + + val testCases = Seq( + JsonToStructsTestCase("{\"a\":1, \"b\":2.0}", "a INT, b DOUBLE", + "UTF8_BINARY", Row(Row(1, 2.0))), + JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_BINARY_LCASE, B INT", + "UTF8_BINARY_LCASE", Row(Row("3", 4))), + JsonToStructsTestCase("{\"x\":true, \"y\":null}", "x BOOLEAN, y VOID", + "UNICODE", Row(Row(true, null))), + JsonToStructsTestCase("{\"X\":null, \"Y\":false}", "X VOID, Y BOOLEAN", + "UNICODE_CI", Row(Row(null, false))) + ) + + // Supported collations + testCases.foreach(t => { + val query = + s""" + |SELECT from_json('${t.input}', '${t.schema}') + |""".stripMargin + // Result & data type + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { + val testQuery = sql(query) + checkAnswer(testQuery, t.result) + val dataType = StructType.fromDDL(t.schema) + assert(testQuery.schema.fields.head.dataType.sameType(dataType)) + } + }) + } + + test("Support StructsToJson json expression with collation") { + case class StructsToJsonTestCase( + struct: String, + collationName: String, + result: Row + ) + + val testCases = Seq( + StructsToJsonTestCase("named_struct('a', 1, 'b', 2)", + "UTF8_BINARY", Row("{\"a\":1,\"b\":2}")), + StructsToJsonTestCase("array(named_struct('a', 1, 'b', 2))", + "UTF8_BINARY_LCASE", Row("[{\"a\":1,\"b\":2}]")), + StructsToJsonTestCase("map('a', named_struct('b', 1))", + "UNICODE", Row("{\"a\":{\"b\":1}}")), + StructsToJsonTestCase("array(map('a', 1))", + "UNICODE_CI", Row("[{\"a\":1}]")) + ) + + // Supported collations + testCases.foreach(t => { + val query = + s""" + |SELECT to_json(${t.struct}) + |""".stripMargin + // Result & data type + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { + val testQuery = sql(query) + checkAnswer(testQuery, t.result) + val dataType = StringType(t.collationName) + assert(testQuery.schema.fields.head.dataType.sameType(dataType)) + } + }) + } + + test("Support LengthOfJsonArray json expression with collation") { + case class LengthOfJsonArrayTestCase( + input: String, + collationName: String, + result: Row + ) + + val testCases = Seq( + LengthOfJsonArrayTestCase("'[1,2,3,4]'", "UTF8_BINARY", Row(4)), + LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_BINARY_LCASE", Row(5)), + LengthOfJsonArrayTestCase("'[1,2'", "UNICODE", Row(null)), + LengthOfJsonArrayTestCase("'['", "UNICODE_CI", Row(null)) + ) + + // Supported collations + testCases.foreach(t => { + val query = + s""" + |SELECT json_array_length(${t.input}) + |""".stripMargin + // Result & data type + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { + val testQuery = sql(query) + checkAnswer(testQuery, t.result) + assert(testQuery.schema.fields.head.dataType.sameType(IntegerType)) + } + }) + } + + test("Support JsonObjectKeys json expression with collation") { + case class JsonObjectKeysJsonArrayTestCase( + input: String, + collationName: String, + result: Row + ) + + val testCases = Seq( + JsonObjectKeysJsonArrayTestCase("{}", "UTF8_BINARY", + Row(Seq())), + JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_BINARY_LCASE", + Row(null)), + JsonObjectKeysJsonArrayTestCase("{\"k1\": \"v1\"}", "UNICODE", + Row(Seq("k1"))), + JsonObjectKeysJsonArrayTestCase("{\"k1\":1,\"k2\":{\"k3\":3, \"k4\":4}}", "UNICODE_CI", + Row(Seq("k1", "k2"))) + ) + + // Supported collations + testCases.foreach(t => { + val query = + s""" + |SELECT json_object_keys('${t.input}') + |""".stripMargin + // Result & data type + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> t.collationName) { + val testQuery = sql(query) + checkAnswer(testQuery, t.result) + val dataType = ArrayType(StringType(t.collationName)) + assert(testQuery.schema.fields.head.dataType.sameType(dataType)) + } + }) + } + test("Support StringToMap expression with collation") { // Supported collations case class StringToMapTestCase[R](t: String, p: String, k: String, c: String, result: R) From 47afe77242abf639a1d6966ce60cfd170a9d7d20 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 8 May 2024 07:44:22 -0700 Subject: [PATCH 12/19] [SPARK-48184][PYTHON][CONNECT] Always set the seed of `Dataframe.sample` in Client side ### What changes were proposed in this pull request? Always set the seed of `Dataframe.sample` in Client side ### Why are the changes needed? Bug fix If the seed is not set in Client, it will be set in server side with a random int https://github.com/apache/spark/blob/c4df12cc884cddefcfcf8324b4d7b9349fb4f6a0/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala#L386 which cause inconsistent results in multiple executions In Spark Classic: ``` In [1]: df = spark.range(10000).sample(0.1) In [2]: [df.count() for i in range(10)] Out[2]: [1006, 1006, 1006, 1006, 1006, 1006, 1006, 1006, 1006, 1006] ``` In Spark Connect: before: ``` In [1]: df = spark.range(10000).sample(0.1) In [2]: [df.count() for i in range(10)] Out[2]: [969, 1005, 958, 996, 987, 1026, 991, 1020, 1012, 979] ``` after: ``` In [1]: df = spark.range(10000).sample(0.1) In [2]: [df.count() for i in range(10)] Out[2]: [1032, 1032, 1032, 1032, 1032, 1032, 1032, 1032, 1032, 1032] ``` ### Does this PR introduce _any_ user-facing change? yes, bug fix ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #46456 from zhengruifeng/py_connect_sample_seed. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/connect/dataframe.py | 2 +- python/pyspark/sql/tests/connect/test_connect_plan.py | 2 +- python/pyspark/sql/tests/test_dataframe.py | 5 +++++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index f9a209d2bcb3d..843c92a9b27d2 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -813,7 +813,7 @@ def sample( if withReplacement is None: withReplacement = False - seed = int(seed) if seed is not None else None + seed = int(seed) if seed is not None else random.randint(0, sys.maxsize) return DataFrame( plan.Sample( diff --git a/python/pyspark/sql/tests/connect/test_connect_plan.py b/python/pyspark/sql/tests/connect/test_connect_plan.py index 09c3171ee11fd..e8d04aeada740 100644 --- a/python/pyspark/sql/tests/connect/test_connect_plan.py +++ b/python/pyspark/sql/tests/connect/test_connect_plan.py @@ -443,7 +443,7 @@ def test_sample(self): self.assertEqual(plan.root.sample.lower_bound, 0.0) self.assertEqual(plan.root.sample.upper_bound, 0.3) self.assertEqual(plan.root.sample.with_replacement, False) - self.assertEqual(plan.root.sample.HasField("seed"), False) + self.assertEqual(plan.root.sample.HasField("seed"), True) self.assertEqual(plan.root.sample.deterministic_order, False) plan = ( diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 16dd0d2a3bf7c..f491b496ddae5 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -430,6 +430,11 @@ def test_sample(self): IllegalArgumentException, lambda: self.spark.range(1).sample(-1.0).count() ) + def test_sample_with_random_seed(self): + df = self.spark.range(10000).sample(0.1) + cnts = [df.count() for i in range(10)] + self.assertEqual(1, len(set(cnts))) + def test_toDF_with_string(self): df = self.spark.createDataFrame([("John", 30), ("Alice", 25), ("Bob", 28)]) data = [("John", 30), ("Alice", 25), ("Bob", 28)] From e0c406eaef36d95a106b6ce14086654ace6202af Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 8 May 2024 08:50:02 -0700 Subject: [PATCH 13/19] [SPARK-48198][BUILD] Upgrade jackson to 2.17.1 ### What changes were proposed in this pull request? The pr aims to upgrade `jackson` from `2.17.0` to `2.17.1`. ### Why are the changes needed? The full release notes: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.17.1 ### 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 #46476 from panbingkun/SPARK-48198. 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 5d933e34e40ba..73d41e9eeb337 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -104,15 +104,15 @@ icu4j/72.1//icu4j-72.1.jar ini4j/0.5.4//ini4j-0.5.4.jar istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar ivy/2.5.2//ivy-2.5.2.jar -jackson-annotations/2.17.0//jackson-annotations-2.17.0.jar +jackson-annotations/2.17.1//jackson-annotations-2.17.1.jar jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar -jackson-core/2.17.0//jackson-core-2.17.0.jar -jackson-databind/2.17.0//jackson-databind-2.17.0.jar -jackson-dataformat-cbor/2.17.0//jackson-dataformat-cbor-2.17.0.jar -jackson-dataformat-yaml/2.17.0//jackson-dataformat-yaml-2.17.0.jar -jackson-datatype-jsr310/2.17.0//jackson-datatype-jsr310-2.17.0.jar +jackson-core/2.17.1//jackson-core-2.17.1.jar +jackson-databind/2.17.1//jackson-databind-2.17.1.jar +jackson-dataformat-cbor/2.17.1//jackson-dataformat-cbor-2.17.1.jar +jackson-dataformat-yaml/2.17.1//jackson-dataformat-yaml-2.17.1.jar +jackson-datatype-jsr310/2.17.1//jackson-datatype-jsr310-2.17.1.jar jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.13/2.17.0//jackson-module-scala_2.13-2.17.0.jar +jackson-module-scala_2.13/2.17.1//jackson-module-scala_2.13-2.17.1.jar jakarta.annotation-api/2.0.0//jakarta.annotation-api-2.0.0.jar jakarta.inject-api/2.0.1//jakarta.inject-api-2.0.1.jar jakarta.servlet-api/5.0.0//jakarta.servlet-api-5.0.0.jar diff --git a/pom.xml b/pom.xml index c72482fd6a41f..c3ff5d101c224 100644 --- a/pom.xml +++ b/pom.xml @@ -183,8 +183,8 @@ true true 1.9.13 - 2.17.0 - 2.17.0 + 2.17.1 + 2.17.1 2.3.1 3.0.2 1.1.10.5 From 9d79ab42b127d1a12164cec260bfbd69f6da8b74 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 8 May 2024 09:40:03 -0700 Subject: [PATCH 14/19] [SPARK-48200][INFRA] Split `build_python.yml` into per-version cron jobs ### What changes were proposed in this pull request? This PR aims to split `build_python.yml` into per-version cron jobs. Technically, this includes a revert of SPARK-48149 and choose [the discussed alternative](https://github.com/apache/spark/pull/46407#discussion_r1591586209). - https://github.com/apache/spark/pull/46407 - https://github.com/apache/spark/pull/46454 ### Why are the changes needed? To recover Python CI successfully in ASF INFRA policy. - https://github.com/apache/spark/actions/workflows/build_python.yml ### 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 #46477 from dongjoon-hyun/SPARK-48200. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- ...build_python.yml => build_python_3.10.yml} | 16 +------ .github/workflows/build_python_3.12.yml | 45 +++++++++++++++++++ .github/workflows/build_python_pypy3.9.yml | 45 +++++++++++++++++++ 3 files changed, 92 insertions(+), 14 deletions(-) rename .github/workflows/{build_python.yml => build_python_3.10.yml} (63%) create mode 100644 .github/workflows/build_python_3.12.yml create mode 100644 .github/workflows/build_python_pypy3.9.yml diff --git a/.github/workflows/build_python.yml b/.github/workflows/build_python_3.10.yml similarity index 63% rename from .github/workflows/build_python.yml rename to .github/workflows/build_python_3.10.yml index efa281d6a279c..5ae37fbc9120e 100644 --- a/.github/workflows/build_python.yml +++ b/.github/workflows/build_python_3.10.yml @@ -17,26 +17,14 @@ # under the License. # -# According to https://infra.apache.org/github-actions-policy.html, -# all workflows SHOULD have a job concurrency level less than or equal to 15. -# To do that, we run one python version per cron schedule -name: "Build / Python-only (master, PyPy 3.9/Python 3.10/Python 3.12)" +name: "Build / Python-only (master, Python 3.10)" on: schedule: - - cron: '0 15 * * *' - cron: '0 17 * * *' - - cron: '0 19 * * *' jobs: run-build: - strategy: - fail-fast: false - matrix: - include: - - pyversion: ${{ github.event.schedule == '0 15 * * *' && 'pypy3' }} - - pyversion: ${{ github.event.schedule == '0 17 * * *' && 'python3.10' }} - - pyversion: ${{ github.event.schedule == '0 19 * * *' && 'python3.12' }} permissions: packages: write name: Run @@ -48,7 +36,7 @@ jobs: hadoop: hadoop3 envs: >- { - "PYTHON_TO_TEST": "${{ matrix.pyversion }}" + "PYTHON_TO_TEST": "python3.10" } jobs: >- { diff --git a/.github/workflows/build_python_3.12.yml b/.github/workflows/build_python_3.12.yml new file mode 100644 index 0000000000000..e1fd45a7d8838 --- /dev/null +++ b/.github/workflows/build_python_3.12.yml @@ -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. +# + +name: "Build / Python-only (master, Python 3.12)" + +on: + schedule: + - cron: '0 19 * * *' + +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: >- + { + "PYTHON_TO_TEST": "python3.12" + } + jobs: >- + { + "pyspark": "true", + "pyspark-pandas": "true" + } diff --git a/.github/workflows/build_python_pypy3.9.yml b/.github/workflows/build_python_pypy3.9.yml new file mode 100644 index 0000000000000..e05071ef034a0 --- /dev/null +++ b/.github/workflows/build_python_pypy3.9.yml @@ -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. +# + +name: "Build / Python-only (master, PyPy 3.9)" + +on: + schedule: + - cron: '0 15 * * *' + +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: >- + { + "PYTHON_TO_TEST": "pypy3" + } + jobs: >- + { + "pyspark": "true", + "pyspark-pandas": "true" + } From 70e5d2aa7a992a6f4ff9c7d8e3752ce1d3d488f2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 8 May 2024 10:47:52 -0700 Subject: [PATCH 15/19] [SPARK-48202][INFRA] Spin off `pyspark` tests from `build_branch35.yml` Daily CI ### What changes were proposed in this pull request? This PR aims to create `build_branch35_python.yml` in order to spin off `pyspark` tests from `build_branch35.yml` Daily CI. ### Why are the changes needed? Currently, `build_branch35.yml` creates more than 15 test pipelines concurrently which is beyond of ASF Infra policy. - https://github.com/apache/spark/actions/workflows/build_branch35.yml We had better offload this to `Python only Daily CI` like `master` branch's `Python Only` Daily CI. - https://github.com/apache/spark/actions/workflows/build_python_3.10.yml ### 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 #46479 from dongjoon-hyun/SPARK-48202. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_branch35.yml | 1 - .github/workflows/build_branch35_python.yml | 45 +++++++++++++++++++++ 2 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/build_branch35_python.yml diff --git a/.github/workflows/build_branch35.yml b/.github/workflows/build_branch35.yml index 55616c2f1f017..2ec080d5722c1 100644 --- a/.github/workflows/build_branch35.yml +++ b/.github/workflows/build_branch35.yml @@ -43,7 +43,6 @@ jobs: jobs: >- { "build": "true", - "pyspark": "true", "sparkr": "true", "tpcds-1g": "true", "docker-integration-tests": "true", diff --git a/.github/workflows/build_branch35_python.yml b/.github/workflows/build_branch35_python.yml new file mode 100644 index 0000000000000..1585534d33ba9 --- /dev/null +++ b/.github/workflows/build_branch35_python.yml @@ -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. +# + +name: "Build / Python-only (branch-3.5)" + +on: + schedule: + - cron: '0 11 * * *' + +jobs: + run-build: + permissions: + packages: write + name: Run + uses: ./.github/workflows/build_and_test.yml + if: github.repository == 'apache/spark' + with: + java: 8 + branch: branch-3.5 + hadoop: hadoop3 + envs: >- + { + "PYTHON_TO_TEST": "" + } + jobs: >- + { + "pyspark": "true", + "pyspark-pandas": "true" + } From fbfcd402851ee604789b8ba72a1ee0e67ef5ebe4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 8 May 2024 12:30:12 -0700 Subject: [PATCH 16/19] [SPARK-48203][INFRA] Spin off `pyspark` tests from `build_branch34.yml` Daily CI ### What changes were proposed in this pull request? This PR aims to create `build_branch34_python.yml` in order to spin off `pyspark` tests from `build_branch34.yml` Daily CI. ### Why are the changes needed? Currently, `build_branch34.yml` creates more than 15 test pipelines concurrently which is beyond of ASF Infra policy. - https://github.com/apache/spark/actions/workflows/build_branch35.yml We had better offload this to `Python only Daily CI` like `master` branch's `Python Only` Daily CI. - https://github.com/apache/spark/actions/workflows/build_python_3.10.yml ### 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 #46480 from dongjoon-hyun/SPARK-48203. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_branch34.yml | 1 - .github/workflows/build_branch34_python.yml | 45 +++++++++++++++++++++ 2 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/build_branch34_python.yml diff --git a/.github/workflows/build_branch34.yml b/.github/workflows/build_branch34.yml index 68887970d4d81..deb6c42407970 100644 --- a/.github/workflows/build_branch34.yml +++ b/.github/workflows/build_branch34.yml @@ -43,7 +43,6 @@ jobs: jobs: >- { "build": "true", - "pyspark": "true", "sparkr": "true", "tpcds-1g": "true", "docker-integration-tests": "true", diff --git a/.github/workflows/build_branch34_python.yml b/.github/workflows/build_branch34_python.yml new file mode 100644 index 0000000000000..c109ba2dc7922 --- /dev/null +++ b/.github/workflows/build_branch34_python.yml @@ -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. +# + +name: "Build / Python-only (branch-3.4)" + +on: + schedule: + - cron: '0 9 * * *' + +jobs: + run-build: + permissions: + packages: write + name: Run + uses: ./.github/workflows/build_and_test.yml + if: github.repository == 'apache/spark' + with: + java: 8 + branch: branch-3.4 + hadoop: hadoop3 + envs: >- + { + "PYTHON_TO_TEST": "" + } + jobs: >- + { + "pyspark": "true", + "pyspark-pandas": "true" + } From 21548a8cc5c527d4416a276a852f967b4410bd4b Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Wed, 8 May 2024 15:44:02 -0400 Subject: [PATCH 17/19] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client ### What changes were proposed in this pull request? This PR adds support for `Dataset.observe` to the Spark Connect Scala client. Note that the support here does not include listener support as it runs on the serve side. This PR includes a small refactoring to the `Observation` helper class. We extracted methods that are not bound to the SparkSession to `spark-api`, and added two subclasses on both `spark-core` and `spark-jvm-client`. ### Why are the changes needed? Before this PR, the `DF.observe` method is only supported in the Python client. ### Does this PR introduce _any_ user-facing change? Yes. The user can now issue `DF.observe(name, metrics...)` or `DF.observe(observationObject, metrics...)` to get stats of columns of a dataframe. ### How was this patch tested? Added new e2e tests. ### Was this patch authored or co-authored using generative AI tooling? Nope. Closes #45701 from xupefei/scala-observe. Authored-by: Paddy Xu Signed-off-by: Herman van Hovell --- .../scala/org/apache/spark/sql/Dataset.scala | 63 +++++- .../org/apache/spark/sql/Observation.scala | 46 +++++ .../org/apache/spark/sql/SparkSession.scala | 31 ++- .../apache/spark/sql/ClientE2ETestSuite.scala | 43 ++++ .../CheckConnectJvmClientCompatibility.scala | 3 - .../main/protobuf/spark/connect/base.proto | 1 + .../sql/connect/client/SparkResult.scala | 44 +++- .../common/LiteralValueProtoConverter.scala | 2 +- .../execution/ExecuteThreadRunner.scala | 1 + .../execution/SparkConnectPlanExecution.scala | 12 +- python/pyspark/sql/connect/proto/base_pb2.py | 188 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 5 +- .../apache/spark/sql/ObservationBase.scala | 113 +++++++++++ .../org/apache/spark/sql/Observation.scala | 62 +----- 14 files changed, 448 insertions(+), 166 deletions(-) create mode 100644 connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.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 9a42afebf8f2b..37f770319b695 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 @@ -3337,8 +3337,69 @@ class Dataset[T] private[sql] ( } } + /** + * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset + * that returns the same result as the input, with the following guarantees:
  • It will + * compute the defined aggregates (metrics) on all the data that is flowing through the Dataset + * at that point.
  • It will report the value of the defined aggregate columns as soon as + * we reach a completion point. A completion point is currently defined as the end of a + * query.
Please note that continuous execution is currently not supported. + * + * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or + * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that + * contain references to the input Dataset's columns must always be wrapped in an aggregate + * function. + * + * A user can retrieve the metrics by calling + * `org.apache.spark.sql.Dataset.collectResult().getObservedMetrics`. + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observed_ds = ds.observe("my_metrics", count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observed_ds.collectResult().getObservedMetrics + * }}} + * + * @group typedrel + * @since 4.0.0 + */ + @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { - throw new UnsupportedOperationException("observe is not implemented.") + sparkSession.newDataset(agnosticEncoder) { builder => + builder.getCollectMetricsBuilder + .setInput(plan.getRoot) + .setName(name) + .addAllMetrics((expr +: exprs).map(_.expr).asJava) + } + } + + /** + * Observe (named) metrics through an `org.apache.spark.sql.Observation` instance. This is + * equivalent to calling `observe(String, Column, Column*)` but does not require to collect all + * results before returning the metrics - the metrics are filled during iterating the results, + * as soon as they are available. This method does not support streaming datasets. + * + * A user can retrieve the metrics by accessing `org.apache.spark.sql.Observation.get`. + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observation = Observation("my_metrics") + * val observed_ds = ds.observe(observation, count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observation.get + * }}} + * + * @throws IllegalArgumentException + * If this is a streaming Dataset (this.isStreaming == true) + * + * @group typedrel + * @since 4.0.0 + */ + @scala.annotation.varargs + def observe(observation: Observation, expr: Column, exprs: Column*): Dataset[T] = { + val df = observe(observation.name, expr, exprs: _*) + sparkSession.registerObservation(df.getPlanId.get, observation) + df } def checkpoint(): Dataset[T] = { diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala new file mode 100644 index 0000000000000..75629b6000f91 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.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 + +import java.util.UUID + +class Observation(name: String) extends ObservationBase(name) { + + /** + * Create an Observation instance without providing a name. This generates a random name. + */ + def this() = this(UUID.randomUUID().toString) +} + +/** + * (Scala-specific) Create instances of Observation via Scala `apply`. + * @since 4.0.0 + */ +object Observation { + + /** + * Observation constructor for creating an anonymous observation. + */ + def apply(): Observation = new Observation() + + /** + * Observation constructor for creating a named observation. + */ + def apply(name: String): Observation = new Observation(name) + +} 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 22bb62803fac5..1188fba60a2fe 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.io.Closeable import java.net.URI +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.TimeUnit._ import java.util.concurrent.atomic.{AtomicLong, AtomicReference} @@ -36,7 +37,7 @@ import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BoxedLongEncoder, UnboundRowEncoder} -import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, SparkResult} +import org.apache.spark.sql.connect.client.{ClassFinder, CloseableIterator, SparkConnectClient, SparkResult} 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 @@ -80,6 +81,8 @@ class SparkSession private[sql] ( client.analyze(proto.AnalyzePlanRequest.AnalyzeCase.SPARK_VERSION).getSparkVersion.getVersion } + private[sql] val observationRegistry = new ConcurrentHashMap[Long, Observation]() + /** * Runtime configuration interface for Spark. * @@ -532,8 +535,12 @@ class SparkSession private[sql] ( private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) - val result = new SparkResult(value, allocator, encoder, timeZoneId) - result + new SparkResult( + value, + allocator, + encoder, + timeZoneId, + Some(setMetricsAndUnregisterObservation)) } private[sql] def execute(f: proto.Relation.Builder => Unit): Unit = { @@ -554,6 +561,9 @@ class SparkSession private[sql] ( client.execute(plan).filter(!_.hasExecutionProgress).toSeq } + private[sql] def execute(plan: proto.Plan): CloseableIterator[ExecutePlanResponse] = + client.execute(plan) + private[sql] def registerUdf(udf: proto.CommonInlineUserDefinedFunction): Unit = { val command = proto.Command.newBuilder().setRegisterFunction(udf).build() execute(command) @@ -779,6 +789,21 @@ class SparkSession private[sql] ( * Set to false to prevent client.releaseSession on close() (testing only) */ private[sql] var releaseSessionOnClose = true + + private[sql] def registerObservation(planId: Long, observation: Observation): Unit = { + if (observationRegistry.putIfAbsent(planId, observation) != null) { + throw new IllegalArgumentException("An Observation can be used with a Dataset only once") + } + } + + private[sql] def setMetricsAndUnregisterObservation( + planId: Long, + metrics: Map[String, Any]): Unit = { + val observationOrNull = observationRegistry.remove(planId) + if (observationOrNull != null) { + observationOrNull.setMetricsAndNotify(Some(metrics)) + } + } } // The minimal builder needed to create a spark session. 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 a0729adb89609..73a2f6d4f88e1 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 @@ -22,6 +22,8 @@ import java.time.DateTimeException import java.util.Properties import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.DurationInt import scala.jdk.CollectionConverters._ import org.apache.commons.io.FileUtils @@ -41,6 +43,7 @@ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.test.{IntegrationTestUtils, RemoteSparkSession, SQLHelper} import org.apache.spark.sql.test.SparkConnectServerUtils.port import org.apache.spark.sql.types._ +import org.apache.spark.util.SparkThreadUtils class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateMethodTester { @@ -1511,6 +1514,46 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM (0 until 5).foreach(i => assert(row.get(i * 2) === row.get(i * 2 + 1))) } } + + test("Observable metrics") { + val df = spark.range(99).withColumn("extra", col("id") - 1) + val ob1 = new Observation("ob1") + val observedDf = df.observe(ob1, min("id"), avg("id"), max("id")) + val observedObservedDf = observedDf.observe("ob2", min("extra"), avg("extra"), max("extra")) + + val ob1Schema = new StructType() + .add("min(id)", LongType) + .add("avg(id)", DoubleType) + .add("max(id)", LongType) + val ob2Schema = new StructType() + .add("min(extra)", LongType) + .add("avg(extra)", DoubleType) + .add("max(extra)", LongType) + val ob1Metrics = Map("ob1" -> new GenericRowWithSchema(Array(0, 49, 98), ob1Schema)) + val ob2Metrics = Map("ob2" -> new GenericRowWithSchema(Array(-1, 48, 97), ob2Schema)) + + 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(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)) + } } private[sql] case class ClassData(a: String, b: Int) 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 c89dba03ed699..7be5e2ecd1725 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 @@ -196,9 +196,6 @@ object CheckConnectJvmClientCompatibility { ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.COL_POS_KEY"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.DATASET_ID_KEY"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.curId"), - ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.observe"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener$"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.queryExecution"), diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 49a33d3419b6f..77dda277602ab 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -434,6 +434,7 @@ message ExecutePlanResponse { string name = 1; repeated Expression.Literal values = 2; repeated string keys = 3; + int64 plan_id = 4; } message ResultComplete { diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 93d1075aea025..0905ee76c3f34 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -27,10 +27,13 @@ import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch} import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator} -import org.apache.spark.sql.connect.common.DataTypeProtoConverter +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter} import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.ArrowUtils @@ -38,7 +41,8 @@ private[sql] class SparkResult[T]( responses: CloseableIterator[proto.ExecutePlanResponse], allocator: BufferAllocator, encoder: AgnosticEncoder[T], - timeZoneId: String) + timeZoneId: String, + setObservationMetricsOpt: Option[(Long, Map[String, Any]) => Unit] = None) extends AutoCloseable { self => case class StageInfo( @@ -79,6 +83,7 @@ private[sql] class SparkResult[T]( private[this] var arrowSchema: pojo.Schema = _ private[this] var nextResultIndex: Int = 0 private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])] + private val observedMetrics = mutable.Map.empty[String, Row] private val cleanable = SparkResult.cleaner.register(this, new SparkResultCloseable(resultMap, responses)) @@ -117,6 +122,9 @@ private[sql] class SparkResult[T]( while (!stop && responses.hasNext) { val response = responses.next() + // Collect metrics for this response + observedMetrics ++= processObservedMetrics(response.getObservedMetricsList) + // Save and validate operationId if (opId == null) { opId = response.getOperationId @@ -198,6 +206,29 @@ private[sql] class SparkResult[T]( nonEmpty } + 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 keys = mutable.ListBuffer.empty[String] + val values = mutable.ListBuffer.empty[Any] + (0 until metric.getKeysCount).map { i => + val key = metric.getKeys(i) + val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) + schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass)) + keys += key + values += value + } + // If the metrics is registered by an Observation object, attach them and unblock any + // blocked thread. + setObservationMetricsOpt.foreach { setObservationMetrics => + setObservationMetrics(metric.getPlanId, keys.zip(values).toMap) + } + metric.getName -> new GenericRowWithSchema(values.toArray, schema) + } + } + /** * Returns the number of elements in the result. */ @@ -248,6 +279,15 @@ private[sql] class SparkResult[T]( result } + /** + * Returns all observed metrics in the result. + */ + def getObservedMetrics: Map[String, Row] = { + // We need to process all responses to get all metrics. + processResponses() + observedMetrics.toMap + } + /** * Returns an iterator over the contents of the result. */ diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala index ce42cc797bf38..1f3496fa89847 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala @@ -204,7 +204,7 @@ object LiteralValueProtoConverter { def toLiteralProto(literal: Any, dataType: DataType): proto.Expression.Literal = toLiteralProtoBuilder(literal, dataType).build() - private def toDataType(clz: Class[_]): DataType = clz match { + private[sql] def toDataType(clz: Class[_]): DataType = clz match { // primitive types case JShort.TYPE => ShortType case JInteger.TYPE => IntegerType diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index 0a6d12cbb1918..4ef4f632204b3 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -220,6 +220,7 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends .createObservedMetricsResponse( executeHolder.sessionHolder.sessionId, executeHolder.sessionHolder.serverSessionId, + executeHolder.request.getPlan.getRoot.getCommon.getPlanId, observedMetrics ++ accumulatedInPython)) } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index 4f2b8c945127b..660951f229849 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -264,8 +264,14 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) name -> values } if (observedMetrics.nonEmpty) { - Some(SparkConnectPlanExecution - .createObservedMetricsResponse(sessionId, sessionHolder.serverSessionId, observedMetrics)) + val planId = executeHolder.request.getPlan.getRoot.getCommon.getPlanId + Some( + SparkConnectPlanExecution + .createObservedMetricsResponse( + sessionId, + sessionHolder.serverSessionId, + planId, + observedMetrics)) } else None } } @@ -274,11 +280,13 @@ object SparkConnectPlanExecution { def createObservedMetricsResponse( sessionId: String, serverSessionId: String, + planId: 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) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 2a30ffe60a9f2..a39396db4ff1d 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -37,7 +37,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\x15\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\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\x1at\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\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_idx2\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"\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"\x80\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\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_idx2\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' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -120,7 +120,7 @@ _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_start = 5196 _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_end = 5361 _EXECUTEPLANRESPONSE._serialized_start = 5440 - _EXECUTEPLANRESPONSE._serialized_end = 8230 + _EXECUTEPLANRESPONSE._serialized_end = 8256 _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_start = 7030 _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_end = 7101 _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 7103 @@ -133,96 +133,96 @@ _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 7651 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 7653 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 7741 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7743 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7859 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7861 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7877 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_start = 7880 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_end = 8213 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_start = 8036 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_end = 8213 - _KEYVALUE._serialized_start = 8232 - _KEYVALUE._serialized_end = 8297 - _CONFIGREQUEST._serialized_start = 8300 - _CONFIGREQUEST._serialized_end = 9459 - _CONFIGREQUEST_OPERATION._serialized_start = 8608 - _CONFIGREQUEST_OPERATION._serialized_end = 9106 - _CONFIGREQUEST_SET._serialized_start = 9108 - _CONFIGREQUEST_SET._serialized_end = 9160 - _CONFIGREQUEST_GET._serialized_start = 9162 - _CONFIGREQUEST_GET._serialized_end = 9187 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 9189 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 9252 - _CONFIGREQUEST_GETOPTION._serialized_start = 9254 - _CONFIGREQUEST_GETOPTION._serialized_end = 9285 - _CONFIGREQUEST_GETALL._serialized_start = 9287 - _CONFIGREQUEST_GETALL._serialized_end = 9335 - _CONFIGREQUEST_UNSET._serialized_start = 9337 - _CONFIGREQUEST_UNSET._serialized_end = 9364 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 9366 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 9400 - _CONFIGRESPONSE._serialized_start = 9462 - _CONFIGRESPONSE._serialized_end = 9637 - _ADDARTIFACTSREQUEST._serialized_start = 9640 - _ADDARTIFACTSREQUEST._serialized_end = 10642 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 10115 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 10168 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 10170 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 10281 - _ADDARTIFACTSREQUEST_BATCH._serialized_start = 10283 - _ADDARTIFACTSREQUEST_BATCH._serialized_end = 10376 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 10379 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 10572 - _ADDARTIFACTSRESPONSE._serialized_start = 10645 - _ADDARTIFACTSRESPONSE._serialized_end = 10917 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 10836 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 10917 - _ARTIFACTSTATUSESREQUEST._serialized_start = 10920 - _ARTIFACTSTATUSESREQUEST._serialized_end = 11246 - _ARTIFACTSTATUSESRESPONSE._serialized_start = 11249 - _ARTIFACTSTATUSESRESPONSE._serialized_end = 11601 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 11444 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 11559 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 11561 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 11601 - _INTERRUPTREQUEST._serialized_start = 11604 - _INTERRUPTREQUEST._serialized_end = 12207 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 12007 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 12135 - _INTERRUPTRESPONSE._serialized_start = 12210 - _INTERRUPTRESPONSE._serialized_end = 12354 - _REATTACHOPTIONS._serialized_start = 12356 - _REATTACHOPTIONS._serialized_end = 12409 - _REATTACHEXECUTEREQUEST._serialized_start = 12412 - _REATTACHEXECUTEREQUEST._serialized_end = 12818 - _RELEASEEXECUTEREQUEST._serialized_start = 12821 - _RELEASEEXECUTEREQUEST._serialized_end = 13406 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 13275 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 13287 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 13289 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 13336 - _RELEASEEXECUTERESPONSE._serialized_start = 13409 - _RELEASEEXECUTERESPONSE._serialized_end = 13574 - _RELEASESESSIONREQUEST._serialized_start = 13577 - _RELEASESESSIONREQUEST._serialized_end = 13748 - _RELEASESESSIONRESPONSE._serialized_start = 13750 - _RELEASESESSIONRESPONSE._serialized_end = 13858 - _FETCHERRORDETAILSREQUEST._serialized_start = 13861 - _FETCHERRORDETAILSREQUEST._serialized_end = 14193 - _FETCHERRORDETAILSRESPONSE._serialized_start = 14196 - _FETCHERRORDETAILSRESPONSE._serialized_end = 15751 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 14425 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 14599 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 14602 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 14970 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 14933 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 14970 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 14973 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 15382 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 15284 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 15352 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 15385 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 15732 - _SPARKCONNECTSERVICE._serialized_start = 15754 - _SPARKCONNECTSERVICE._serialized_end = 16700 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7744 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7885 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7887 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7903 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_start = 7906 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_end = 8239 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_start = 8062 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_end = 8239 + _KEYVALUE._serialized_start = 8258 + _KEYVALUE._serialized_end = 8323 + _CONFIGREQUEST._serialized_start = 8326 + _CONFIGREQUEST._serialized_end = 9485 + _CONFIGREQUEST_OPERATION._serialized_start = 8634 + _CONFIGREQUEST_OPERATION._serialized_end = 9132 + _CONFIGREQUEST_SET._serialized_start = 9134 + _CONFIGREQUEST_SET._serialized_end = 9186 + _CONFIGREQUEST_GET._serialized_start = 9188 + _CONFIGREQUEST_GET._serialized_end = 9213 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 9215 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 9278 + _CONFIGREQUEST_GETOPTION._serialized_start = 9280 + _CONFIGREQUEST_GETOPTION._serialized_end = 9311 + _CONFIGREQUEST_GETALL._serialized_start = 9313 + _CONFIGREQUEST_GETALL._serialized_end = 9361 + _CONFIGREQUEST_UNSET._serialized_start = 9363 + _CONFIGREQUEST_UNSET._serialized_end = 9390 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 9392 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 9426 + _CONFIGRESPONSE._serialized_start = 9488 + _CONFIGRESPONSE._serialized_end = 9663 + _ADDARTIFACTSREQUEST._serialized_start = 9666 + _ADDARTIFACTSREQUEST._serialized_end = 10668 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 10141 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 10194 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 10196 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 10307 + _ADDARTIFACTSREQUEST_BATCH._serialized_start = 10309 + _ADDARTIFACTSREQUEST_BATCH._serialized_end = 10402 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 10405 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 10598 + _ADDARTIFACTSRESPONSE._serialized_start = 10671 + _ADDARTIFACTSRESPONSE._serialized_end = 10943 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 10862 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 10943 + _ARTIFACTSTATUSESREQUEST._serialized_start = 10946 + _ARTIFACTSTATUSESREQUEST._serialized_end = 11272 + _ARTIFACTSTATUSESRESPONSE._serialized_start = 11275 + _ARTIFACTSTATUSESRESPONSE._serialized_end = 11627 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 11470 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 11585 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 11587 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 11627 + _INTERRUPTREQUEST._serialized_start = 11630 + _INTERRUPTREQUEST._serialized_end = 12233 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 12033 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 12161 + _INTERRUPTRESPONSE._serialized_start = 12236 + _INTERRUPTRESPONSE._serialized_end = 12380 + _REATTACHOPTIONS._serialized_start = 12382 + _REATTACHOPTIONS._serialized_end = 12435 + _REATTACHEXECUTEREQUEST._serialized_start = 12438 + _REATTACHEXECUTEREQUEST._serialized_end = 12844 + _RELEASEEXECUTEREQUEST._serialized_start = 12847 + _RELEASEEXECUTEREQUEST._serialized_end = 13432 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 13301 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 13313 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 13315 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 13362 + _RELEASEEXECUTERESPONSE._serialized_start = 13435 + _RELEASEEXECUTERESPONSE._serialized_end = 13600 + _RELEASESESSIONREQUEST._serialized_start = 13603 + _RELEASESESSIONREQUEST._serialized_end = 13774 + _RELEASESESSIONRESPONSE._serialized_start = 13776 + _RELEASESESSIONRESPONSE._serialized_end = 13884 + _FETCHERRORDETAILSREQUEST._serialized_start = 13887 + _FETCHERRORDETAILSREQUEST._serialized_end = 14219 + _FETCHERRORDETAILSRESPONSE._serialized_start = 14222 + _FETCHERRORDETAILSRESPONSE._serialized_end = 15777 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 14451 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 14625 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 14628 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 14996 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 14959 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 14996 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 14999 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 15408 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 15310 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 15378 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 15411 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 15758 + _SPARKCONNECTSERVICE._serialized_start = 15780 + _SPARKCONNECTSERVICE._serialized_end = 16726 # @@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 d22502f8839db..b76f2a7f4de34 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -1406,6 +1406,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): NAME_FIELD_NUMBER: builtins.int VALUES_FIELD_NUMBER: builtins.int KEYS_FIELD_NUMBER: builtins.int + PLAN_ID_FIELD_NUMBER: builtins.int name: builtins.str @property def values( @@ -1417,6 +1418,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): def keys( self, ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: ... + plan_id: builtins.int def __init__( self, *, @@ -1426,11 +1428,12 @@ class ExecutePlanResponse(google.protobuf.message.Message): ] | None = ..., keys: collections.abc.Iterable[builtins.str] | None = ..., + plan_id: builtins.int = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "keys", b"keys", "name", b"name", "values", b"values" + "keys", b"keys", "name", b"name", "plan_id", b"plan_id", "values", b"values" ], ) -> None: ... diff --git a/sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala b/sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala new file mode 100644 index 0000000000000..4789ae8975d12 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala @@ -0,0 +1,113 @@ +/* + * 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 scala.jdk.CollectionConverters.MapHasAsJava + +/** + * Helper class to simplify usage of `Dataset.observe(String, Column, Column*)`: + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observation = Observation("my metrics") + * val observed_ds = ds.observe(observation, count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observation.get + * }}} + * + * This collects the metrics while the first action is executed on the observed dataset. Subsequent + * actions do not modify the metrics returned by [[get]]. Retrieval of the metric via [[get]] + * blocks until the first action has finished and metrics become available. + * + * This class does not support streaming datasets. + * + * @param name name of the metric + * @since 3.3.0 + */ +abstract class ObservationBase(val name: String) { + + if (name.isEmpty) throw new IllegalArgumentException("Name must not be empty") + + @volatile protected var metrics: Option[Map[String, Any]] = None + + /** + * (Scala-specific) Get the observed metrics. This waits for the observed dataset to finish + * its first action. Only the result of the first action is available. Subsequent actions do not + * modify the result. + * + * @return the observed metrics as a `Map[String, Any]` + * @throws InterruptedException interrupted while waiting + */ + @throws[InterruptedException] + def get: Map[String, _] = { + synchronized { + // we need to loop as wait might return without us calling notify + // https://en.wikipedia.org/w/index.php?title=Spurious_wakeup&oldid=992601610 + while (this.metrics.isEmpty) { + wait() + } + } + + this.metrics.get + } + + /** + * (Java-specific) Get the observed metrics. This waits for the observed dataset to finish + * its first action. Only the result of the first action is available. Subsequent actions do not + * modify the result. + * + * @return the observed metrics as a `java.util.Map[String, Object]` + * @throws InterruptedException interrupted while waiting + */ + @throws[InterruptedException] + def getAsJava: java.util.Map[String, AnyRef] = { + get.map { case (key, value) => (key, value.asInstanceOf[Object]) }.asJava + } + + /** + * Get the observed metrics. This returns the metrics if they are available, otherwise an empty. + * + * @return the observed metrics as a `Map[String, Any]` + */ + @throws[InterruptedException] + private[sql] def getOrEmpty: Map[String, _] = { + synchronized { + if (metrics.isEmpty) { + wait(100) // Wait for 100ms to see if metrics are available + } + metrics.getOrElse(Map.empty) + } + } + + /** + * Set the observed metrics and notify all waiting threads to resume. + * + * @return `true` if all waiting threads were notified, `false` if otherwise. + */ + private[spark] def setMetricsAndNotify(metrics: Option[Map[String, Any]]): Boolean = { + synchronized { + this.metrics = metrics + if(metrics.isDefined) { + notifyAll() + true + } else { + false + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala b/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala index 104e7c101fd1c..30d5943c60922 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql import java.util.UUID -import scala.jdk.CollectionConverters.MapHasAsJava - import org.apache.spark.sql.catalyst.plans.logical.CollectMetrics import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.util.QueryExecutionListener @@ -47,9 +45,7 @@ import org.apache.spark.util.ArrayImplicits._ * @param name name of the metric * @since 3.3.0 */ -class Observation(val name: String) { - - if (name.isEmpty) throw new IllegalArgumentException("Name must not be empty") +class Observation(name: String) extends ObservationBase(name) { /** * Create an Observation instance without providing a name. This generates a random name. @@ -60,8 +56,6 @@ class Observation(val name: String) { @volatile private var dataframeId: Option[(SparkSession, Long)] = None - @volatile private var metrics: Option[Map[String, Any]] = None - /** * Attach this observation to the given [[Dataset]] to observe aggregation expressions. * @@ -83,55 +77,6 @@ class Observation(val name: String) { ds.observe(name, expr, exprs: _*) } - /** - * (Scala-specific) Get the observed metrics. This waits for the observed dataset to finish - * its first action. Only the result of the first action is available. Subsequent actions do not - * modify the result. - * - * @return the observed metrics as a `Map[String, Any]` - * @throws InterruptedException interrupted while waiting - */ - @throws[InterruptedException] - def get: Map[String, _] = { - synchronized { - // we need to loop as wait might return without us calling notify - // https://en.wikipedia.org/w/index.php?title=Spurious_wakeup&oldid=992601610 - while (this.metrics.isEmpty) { - wait() - } - } - - this.metrics.get - } - - /** - * (Java-specific) Get the observed metrics. This waits for the observed dataset to finish - * its first action. Only the result of the first action is available. Subsequent actions do not - * modify the result. - * - * @return the observed metrics as a `java.util.Map[String, Object]` - * @throws InterruptedException interrupted while waiting - */ - @throws[InterruptedException] - def getAsJava: java.util.Map[String, AnyRef] = { - get.map { case (key, value) => (key, value.asInstanceOf[Object])}.asJava - } - - /** - * Get the observed metrics. This returns the metrics if they are available, otherwise an empty. - * - * @return the observed metrics as a `Map[String, Any]` - */ - @throws[InterruptedException] - private[sql] def getOrEmpty: Map[String, _] = { - synchronized { - if (metrics.isEmpty) { - wait(100) // Wait for 100ms to see if metrics are available - } - metrics.getOrElse(Map.empty) - } - } - private[sql] def register(sparkSession: SparkSession, dataframeId: Long): Unit = { // makes this class thread-safe: // only the first thread entering this block can set sparkSession @@ -158,9 +103,8 @@ class Observation(val name: String) { case _ => false }) { val row = qe.observedMetrics.get(name) - this.metrics = row.map(r => r.getValuesMap[Any](r.schema.fieldNames.toImmutableArraySeq)) - if (metrics.isDefined) { - notifyAll() + val metrics = row.map(r => r.getValuesMap[Any](r.schema.fieldNames.toImmutableArraySeq)) + if (setMetricsAndNotify(metrics)) { unregister() } } From 4fb6624bd2cec0fec893ea0ac65b1a02c60384ec Mon Sep 17 00:00:00 2001 From: allisonwang-db Date: Thu, 9 May 2024 08:22:48 +0900 Subject: [PATCH 18/19] [SPARK-48205][PYTHON] Remove the private[sql] modifier for Python data sources ### What changes were proposed in this pull request? This PR removes the `private[sql]` modifier for Python data sources to make it consistent with UDFs and UDTFs. ### Why are the changes needed? ### 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 #46487 from allisonwang-db/spark-48205-pyds-modifier. Authored-by: allisonwang-db Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/DataSourceRegistration.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala index 63cee8861c5a4..8ffdbb952b082 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.internal.SQLConf * Use `SparkSession.dataSource` to access this. */ @Evolving -private[sql] class DataSourceRegistration private[sql] (dataSourceManager: DataSourceManager) +class DataSourceRegistration private[sql] (dataSourceManager: DataSourceManager) extends Logging { protected[sql] def registerPython( 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 5d85f070fbbe4..d5de74455dceb 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 @@ -234,7 +234,7 @@ class SparkSession private( /** * A collection of methods for registering user-defined data sources. */ - private[sql] def dataSource: DataSourceRegistration = sessionState.dataSourceRegistration + def dataSource: DataSourceRegistration = sessionState.dataSourceRegistration /** * Returns a `StreamingQueryManager` that allows managing all the From 337f980f0073c8605ed2738186d2089a362b7f66 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 May 2024 16:47:01 -0700 Subject: [PATCH 19/19] [SPARK-48204][INFRA] Fix release script for Spark 4.0+ ### What changes were proposed in this pull request? Before Spark 4.0, Scala 2.12 was primary and Scala 2.13 was secondary. The release scripts build more packages (hadoop3, without-hadoop, pyspark, sparkr) for the primary Scala version but only one package for the secondary. However, Spark 4.0 removes Scala 2.12 support and the release script needs to be updated accordingly. ### Why are the changes needed? to make the release scripts work ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? manual. ### Was this patch authored or co-authored using generative AI tooling? no Closes #46484 from cloud-fan/re. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 75ec98464f3ec..b720a8fc93861 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -194,6 +194,8 @@ fi PUBLISH_SCALA_2_12=1 if [[ $SPARK_VERSION > "3.5.99" ]]; then PUBLISH_SCALA_2_12=0 + # There is no longer scala-2.13 profile since 4.0.0 + SCALA_2_13_PROFILES="" fi SCALA_2_12_PROFILES="-Pscala-2.12" @@ -345,21 +347,25 @@ if [[ "$1" == "package" ]]; then declare -A BINARY_PKGS_EXTRA BINARY_PKGS_EXTRA["hadoop3"]="withpip,withr" - if [[ $PUBLISH_SCALA_2_13 = 1 ]]; then - key="hadoop3-scala2.13" + # This is dead code as Scala 2.12 is no longer supported, but we keep it as a template for + # adding new Scala version support in the future. This secondary Scala version only has one + # binary package to avoid doubling the number of final packages. It doesn't build PySpark and + # SparkR as the primary Scala version will build them. + if [[ $PUBLISH_SCALA_2_12 = 1 ]]; then + key="hadoop3-scala2.12" args="-Phadoop-3 $HIVE_PROFILES" extra="" - if ! make_binary_release "$key" "$SCALA_2_13_PROFILES $args" "$extra" "2.13"; then + if ! make_binary_release "$key" "$SCALA_2_12_PROFILES $args" "$extra" "2.12"; then error "Failed to build $key package. Check logs for details." fi fi - if [[ $PUBLISH_SCALA_2_12 = 1 ]]; then + if [[ $PUBLISH_SCALA_2_13 = 1 ]]; then echo "Packages to build: ${!BINARY_PKGS_ARGS[@]}" for key in ${!BINARY_PKGS_ARGS[@]}; do args=${BINARY_PKGS_ARGS[$key]} extra=${BINARY_PKGS_EXTRA[$key]} - if ! make_binary_release "$key" "$SCALA_2_12_PROFILES $args" "$extra" "2.12"; then + if ! make_binary_release "$key" "$SCALA_2_13_PROFILES $args" "$extra" "2.13"; then error "Failed to build $key package. Check logs for details." fi done