From 501999a834ea7761a792b823c543e40fba84231d Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Thu, 7 Mar 2024 13:20:39 -0800 Subject: [PATCH 01/50] [SPARK-47276][PYTHON][CONNECT] Introduce `spark.profile.clear` for SparkSession-based profiling ### What changes were proposed in this pull request? Introduce `spark.profile.clear` for SparkSession-based profiling. ### Why are the changes needed? A straightforward and unified interface for managing and resetting profiling results for SparkSession-based profilers. ### Does this PR introduce _any_ user-facing change? Yes. `spark.profile.clear` is supported as shown below. Preparation: ```py >>> from pyspark.sql.functions import pandas_udf >>> df = spark.range(3) >>> pandas_udf("long") ... def add1(x): ... return x + 1 ... >>> added = df.select(add1("id")) >>> spark.conf.set("spark.sql.pyspark.udf.profiler", "perf") >>> added.show() +--------+ |add1(id)| +--------+ ... +--------+ >>> spark.profile.show() ============================================================ Profile of UDF ============================================================ 1410 function calls (1374 primitive calls) in 0.004 seconds ... ``` Example usage: ```py >>> spark.profile.profiler_collector._profile_results {2: (, None)} >>> spark.profile.clear(1) # id mismatch >>> spark.profile.profiler_collector._profile_results {2: (, None)} >>> spark.profile.clear(type="memory") # type mismatch >>> spark.profile.profiler_collector._profile_results {2: (, None)} >>> spark.profile.clear() # clear all >>> spark.profile.profiler_collector._profile_results {} >>> spark.profile.show() >>> ``` ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45378 from xinrong-meng/profile_clear. Authored-by: Xinrong Meng Signed-off-by: Xinrong Meng --- python/pyspark/sql/profiler.py | 79 +++++++++++++++++++ python/pyspark/sql/tests/test_session.py | 27 +++++++ python/pyspark/sql/tests/test_udf_profiler.py | 26 ++++++ python/pyspark/tests/test_memory_profiler.py | 59 ++++++++++++++ 4 files changed, 191 insertions(+) diff --git a/python/pyspark/sql/profiler.py b/python/pyspark/sql/profiler.py index 5ab27bce2582b..711e39de4723b 100644 --- a/python/pyspark/sql/profiler.py +++ b/python/pyspark/sql/profiler.py @@ -224,6 +224,56 @@ def dump(id: int) -> None: for id in sorted(code_map.keys()): dump(id) + def clear_perf_profiles(self, id: Optional[int] = None) -> None: + """ + Clear the perf profile results. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + id : int, optional + The UDF ID whose profiling results should be cleared. + If not specified, all the results will be cleared. + """ + with self._lock: + if id is not None: + if id in self._profile_results: + perf, mem, *_ = self._profile_results[id] + self._profile_results[id] = (None, mem, *_) + if mem is None: + self._profile_results.pop(id, None) + else: + for id, (perf, mem, *_) in list(self._profile_results.items()): + self._profile_results[id] = (None, mem, *_) + if mem is None: + self._profile_results.pop(id, None) + + def clear_memory_profiles(self, id: Optional[int] = None) -> None: + """ + Clear the memory profile results. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + id : int, optional + The UDF ID whose profiling results should be cleared. + If not specified, all the results will be cleared. + """ + with self._lock: + if id is not None: + if id in self._profile_results: + perf, mem, *_ = self._profile_results[id] + self._profile_results[id] = (perf, None, *_) + if perf is None: + self._profile_results.pop(id, None) + else: + for id, (perf, mem, *_) in list(self._profile_results.items()): + self._profile_results[id] = (perf, None, *_) + if perf is None: + self._profile_results.pop(id, None) + class AccumulatorProfilerCollector(ProfilerCollector): def __init__(self) -> None: @@ -309,3 +359,32 @@ def dump(self, path: str, id: Optional[int] = None, *, type: Optional[str] = Non "allowed_values": str(["perf", "memory"]), }, ) + + def clear(self, id: Optional[int] = None, *, type: Optional[str] = None) -> None: + """ + Clear the profile results. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + id : int, optional + The UDF ID whose profiling results should be cleared. + If not specified, all the results will be cleared. + type : str, optional + The profiler type to clear results for, which can be either "perf" or "memory". + """ + if type == "memory": + self.profiler_collector.clear_memory_profiles(id) + elif type == "perf" or type is None: + self.profiler_collector.clear_perf_profiles(id) + if type is None: # Clear both perf and memory profiles + self.profiler_collector.clear_memory_profiles(id) + else: + raise PySparkValueError( + error_class="VALUE_NOT_ALLOWED", + message_parameters={ + "arg_name": "type", + "allowed_values": str(["perf", "memory"]), + }, + ) diff --git a/python/pyspark/sql/tests/test_session.py b/python/pyspark/sql/tests/test_session.py index b95e9de9e3f30..5f102d770c6ac 100644 --- a/python/pyspark/sql/tests/test_session.py +++ b/python/pyspark/sql/tests/test_session.py @@ -531,6 +531,33 @@ def test_dump_invalid_type(self): }, ) + def test_clear_memory_type(self): + self.profile.clear(type="memory") + self.profiler_collector_mock.clear_memory_profiles.assert_called_once() + self.profiler_collector_mock.clear_perf_profiles.assert_not_called() + + def test_clear_perf_type(self): + self.profile.clear(type="perf") + self.profiler_collector_mock.clear_perf_profiles.assert_called_once() + self.profiler_collector_mock.clear_memory_profiles.assert_not_called() + + def test_clear_no_type(self): + self.profile.clear() + self.profiler_collector_mock.clear_perf_profiles.assert_called_once() + self.profiler_collector_mock.clear_memory_profiles.assert_called_once() + + def test_clear_invalid_type(self): + with self.assertRaises(PySparkValueError) as e: + self.profile.clear(type="invalid") + self.check_error( + exception=e.exception, + error_class="VALUE_NOT_ALLOWED", + message_parameters={ + "arg_name": "type", + "allowed_values": str(["perf", "memory"]), + }, + ) + class SparkExtensionsTest(unittest.TestCase): # These tests are separate because it uses 'spark.sql.extensions' which is diff --git a/python/pyspark/sql/tests/test_udf_profiler.py b/python/pyspark/sql/tests/test_udf_profiler.py index 557b4daa85508..a66503bc02138 100644 --- a/python/pyspark/sql/tests/test_udf_profiler.py +++ b/python/pyspark/sql/tests/test_udf_profiler.py @@ -521,6 +521,32 @@ def summarize(left, right): io.getvalue(), f"2.*{os.path.basename(inspect.getfile(_do_computation))}" ) + def test_perf_profiler_clear(self): + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): + _do_computation(self.spark) + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + for id in self.profile_results: + self.spark.profile.clear(id) + self.assertNotIn(id, self.profile_results) + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): + _do_computation(self.spark) + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + self.spark.profile.clear(type="memory") + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + self.spark.profile.clear(type="perf") + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): + _do_computation(self.spark) + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + self.spark.profile.clear() + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + class UDFProfiler2Tests(UDFProfiler2TestsMixin, ReusedSQLTestCase): def setUp(self) -> None: diff --git a/python/pyspark/tests/test_memory_profiler.py b/python/pyspark/tests/test_memory_profiler.py index f0abdd03e2437..046dd3621c42d 100644 --- a/python/pyspark/tests/test_memory_profiler.py +++ b/python/pyspark/tests/test_memory_profiler.py @@ -221,6 +221,10 @@ def trap_stdout(self): def profile_results(self): return self.spark._profiler_collector._memory_profile_results + @property + def perf_profile_results(self): + return self.spark._profiler_collector._perf_profile_results + def test_memory_profiler_udf(self): _do_computation(self.spark) @@ -571,6 +575,61 @@ def summarize(left, right): io.getvalue(), f"Filename.*{os.path.basename(inspect.getfile(_do_computation))}" ) + def test_memory_profiler_clear(self): + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "memory"}): + _do_computation(self.spark) + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + for id in list(self.profile_results.keys()): + self.spark.profile.clear(id) + self.assertNotIn(id, self.profile_results) + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "memory"}): + _do_computation(self.spark) + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + self.spark.profile.clear(type="perf") + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + self.spark.profile.clear(type="memory") + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "memory"}): + _do_computation(self.spark) + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + self.spark.profile.clear() + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + + def test_profilers_clear(self): + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "memory"}): + _do_computation(self.spark) + with self.sql_conf({"spark.sql.pyspark.udf.profiler": "perf"}): + _do_computation(self.spark) + + self.assertEqual(3, len(self.profile_results), str(list(self.profile_results))) + + # clear a specific memory profile + some_id = next(iter(self.profile_results)) + self.spark.profile.clear(some_id, type="memory") + self.assertEqual(2, len(self.profile_results), str(list(self.profile_results))) + self.assertEqual(3, len(self.perf_profile_results), str(list(self.perf_profile_results))) + + # clear a specific perf profile + some_id = next(iter(self.perf_profile_results)) + self.spark.profile.clear(some_id, type="perf") + self.assertEqual(2, len(self.perf_profile_results), str(list(self.perf_profile_results))) + self.assertEqual(2, len(self.profile_results), str(list(self.profile_results))) + + # clear all memory profiles + self.spark.profile.clear(type="memory") + self.assertEqual(0, len(self.profile_results), str(list(self.profile_results))) + self.assertEqual(2, len(self.perf_profile_results), str(list(self.perf_profile_results))) + + # clear all perf profiles + self.spark.profile.clear(type="perf") + self.assertEqual(0, len(self.perf_profile_results), str(list(self.perf_profile_results))) + class MemoryProfiler2Tests(MemoryProfiler2TestsMixin, ReusedSQLTestCase): def setUp(self) -> None: From 7a5bb5d83551b749a64f38dc937e9b488616cf8d Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Fri, 8 Mar 2024 09:42:40 +0900 Subject: [PATCH 02/50] [SPARK-47078][DOCS][PYTHON] Documentation for SparkSession-based Profilers ### What changes were proposed in this pull request? Documentation for SparkSession-based Profilers. ### Why are the changes needed? For easier user onboarding and better usability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test. Screenshots of built htmls are as shown below. ![image](https://github.com/apache/spark/assets/47337188/3eeea311-9bfb-42bd-9ff0-fafdc7ab8885) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45269 from xinrong-meng/profiler_doc. Authored-by: Xinrong Meng Signed-off-by: Hyukjin Kwon --- python/docs/source/development/debugging.rst | 69 ++++++------------- .../reference/pyspark.sql/spark_session.rst | 1 + python/pyspark/sql/connect/session.py | 2 + python/pyspark/sql/session.py | 12 ++++ 4 files changed, 35 insertions(+), 49 deletions(-) diff --git a/python/docs/source/development/debugging.rst b/python/docs/source/development/debugging.rst index ef848a6e961e1..b0b2c4837ded4 100644 --- a/python/docs/source/development/debugging.rst +++ b/python/docs/source/development/debugging.rst @@ -215,13 +215,10 @@ Python/Pandas UDF ~~~~~~~~~~~~~~~~~ PySpark provides remote `memory_profiler `_ for -Python/Pandas UDFs, which can be enabled by setting ``spark.python.profile.memory`` configuration to ``true``. That -can be used on editors with line numbers such as Jupyter notebooks. An example on a Jupyter notebook is as shown below. - -.. code-block:: bash - - pyspark --conf spark.python.profile.memory=true +Python/Pandas UDFs. That can be used on editors with line numbers such as Jupyter notebooks. UDFs with iterators as inputs/outputs are not supported. +SparkSession-based memory profiler can be enabled by setting the `Runtime SQL configuration `_ +``spark.sql.pyspark.udf.profiler`` to ``memory``. An example on a Jupyter notebook is as shown below. .. code-block:: python @@ -232,10 +229,11 @@ can be used on editors with line numbers such as Jupyter notebooks. An example o def add1(x): return x + 1 + spark.conf.set("spark.sql.pyspark.udf.profiler", "memory") + added = df.select(add1("id")) added.show() - sc.show_profiles() - + spark.profile.show(type="memory") The result profile is as shown below. @@ -258,7 +256,6 @@ The UDF IDs can be seen in the query plan, for example, ``add1(...)#2L`` in ``Ar added.explain() - .. code-block:: text == Physical Plan == @@ -266,8 +263,11 @@ The UDF IDs can be seen in the query plan, for example, ``add1(...)#2L`` in ``Ar +- ArrowEvalPython [add1(id#0L)#2L], [pythonUDF0#11L], 200 +- *(1) Range (0, 10, step=1, splits=16) -This feature is not supported with registered UDFs or UDFs with iterators as inputs/outputs. +We can clear the result memory profile as shown below. + +.. code-block:: python + spark.profile.clear(id=2, type="memory") Identifying Hot Loops (Python Profilers) ---------------------------------------- @@ -306,47 +306,14 @@ regular Python process unless you are running your driver program in another mac 276 0.000 0.000 0.002 0.000 :147(__enter__) ... -Executor Side -~~~~~~~~~~~~~ - -To use this on executor side, PySpark provides remote `Python Profilers `_ for -executor side, which can be enabled by setting ``spark.python.profile`` configuration to ``true``. - -.. code-block:: bash - - pyspark --conf spark.python.profile=true - - -.. code-block:: python - - >>> rdd = sc.parallelize(range(100)).map(str) - >>> rdd.count() - 100 - >>> sc.show_profiles() - ============================================================ - Profile of RDD - ============================================================ - 728 function calls (692 primitive calls) in 0.004 seconds - - Ordered by: internal time, cumulative time - - ncalls tottime percall cumtime percall filename:lineno(function) - 12 0.001 0.000 0.001 0.000 serializers.py:210(load_stream) - 12 0.000 0.000 0.000 0.000 {built-in method _pickle.dumps} - 12 0.000 0.000 0.001 0.000 serializers.py:252(dump_stream) - 12 0.000 0.000 0.001 0.000 context.py:506(f) - ... - Python/Pandas UDF ~~~~~~~~~~~~~~~~~ -To use this on Python/Pandas UDFs, PySpark provides remote `Python Profilers `_ for -Python/Pandas UDFs, which can be enabled by setting ``spark.python.profile`` configuration to ``true``. - -.. code-block:: bash - - pyspark --conf spark.python.profile=true +PySpark provides remote `Python Profilers `_ for +Python/Pandas UDFs. UDFs with iterators as inputs/outputs are not supported. +SparkSession-based performance profiler can be enabled by setting the `Runtime SQL configuration `_ +``spark.sql.pyspark.udf.profiler`` to ``perf``. An example is as shown below. .. code-block:: python @@ -358,6 +325,7 @@ Python/Pandas UDFs, which can be enabled by setting ``spark.python.profile`` con ... >>> added = df.select(add1("id")) + >>> spark.conf.set("spark.sql.pyspark.udf.profiler", "perf") >>> added.show() +--------+ |add1(id)| @@ -365,7 +333,7 @@ Python/Pandas UDFs, which can be enabled by setting ``spark.python.profile`` con ... +--------+ - >>> sc.show_profiles() + >>> spark.profile.show(type="perf") ============================================================ Profile of UDF ============================================================ @@ -390,8 +358,11 @@ The UDF IDs can be seen in the query plan, for example, ``add1(...)#2L`` in ``Ar +- ArrowEvalPython [add1(id#0L)#2L], [pythonUDF0#11L], 200 +- *(1) Range (0, 10, step=1, splits=16) +We can clear the result performance profile as shown below. + +.. code-block:: python -This feature is not supported with registered UDFs. + >>> spark.profile.clear(id=2, type="perf") Common Exceptions / Errors -------------------------- diff --git a/python/docs/source/reference/pyspark.sql/spark_session.rst b/python/docs/source/reference/pyspark.sql/spark_session.rst index f242e4439cf4c..4be343c52140f 100644 --- a/python/docs/source/reference/pyspark.sql/spark_session.rst +++ b/python/docs/source/reference/pyspark.sql/spark_session.rst @@ -49,6 +49,7 @@ See also :class:`SparkSession`. SparkSession.createDataFrame SparkSession.getActiveSession SparkSession.newSession + SparkSession.profile SparkSession.range SparkSession.read SparkSession.readStream diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 6c4a3064edf47..0e05f2743a0d4 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -946,6 +946,8 @@ def _profiler_collector(self) -> ProfilerCollector: def profile(self) -> Profile: return Profile(self._client._profiler_collector) + profile.__doc__ = PySparkSession.profile.__doc__ + SparkSession.__doc__ = PySparkSession.__doc__ diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index f12a5365d86ca..6c80b7f42da45 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -908,6 +908,18 @@ def dataSource(self) -> "DataSourceRegistration": @property def profile(self) -> Profile: + """Returns a :class:`Profile` for performance/memory profiling. + + .. versionadded:: 4.0.0 + + Returns + ------- + :class:`Profile` + + Notes + ----- + Supports Spark Connect. + """ return Profile(self._profiler_collector) def range( From 9cac2bb608c65f7fec19927711aff6dd1c54005a Mon Sep 17 00:00:00 2001 From: Shujing Yang Date: Fri, 8 Mar 2024 09:44:12 +0900 Subject: [PATCH 03/50] [SPARK-47309][SQL][XML] Fix schema inference issues in XML ### What changes were proposed in this pull request? This PR fixes XML schema inference issues: 1. when there's an empty tag 2. when merging schema for NullType ### Why are the changes needed? Fix a bug ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Unit tests. There's a follow-up [PR](https://github.com/apache/spark/pull/45411) that introduces comprehensive tests for schema inference. ### Was this patch authored or co-authored using generative AI tooling? No Closes #45426 from shujingyang-db/fix-xml-schema-inference. Authored-by: Shujing Yang Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala index be5a29d299a3f..b9342c53d020c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/xml/XmlInferSchema.scala @@ -195,7 +195,9 @@ class XmlInferSchema(options: XmlOptions, caseSensitive: Boolean) private def inferField(parser: XMLEventReader): DataType = { parser.peek match { - case _: EndElement => NullType + case _: EndElement => + parser.nextEvent() + NullType case _: StartElement => inferObject(parser) case _: Characters => val structType = inferObject(parser).asInstanceOf[StructType] @@ -450,7 +452,7 @@ class XmlInferSchema(options: XmlOptions, caseSensitive: Boolean) oldTypeOpt match { // If the field name already exists, // merge the type and infer the combined field as an array type if necessary - case Some(oldType) if !oldType.isInstanceOf[ArrayType] && !newType.isInstanceOf[NullType] => + case Some(oldType) if !oldType.isInstanceOf[ArrayType] => ArrayType(compatibleType(caseSensitive, options.valueTag)(oldType, newType)) case Some(oldType) => compatibleType(caseSensitive, options.valueTag)(oldType, newType) From 703d1d8e4be223f1c569833767e2550b76ce03e2 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 8 Mar 2024 10:55:04 +0900 Subject: [PATCH 04/50] [MINOR][INFRA] Make "y/n" consistent within merge script ### What changes were proposed in this pull request? This PR changes the y/n message and condition consistent within merging script. ### Why are the changes needed? For consistency. ``` Would you like to use the modified body? (y/N): y ... Proceed with merging pull request #45426? (y/N): y ... Merge complete (local ref PR_TOOL_MERGE_PR_45426_MASTER). Push to apache? (y/N): y ... Would you like to pick 9cac2bb6 into another branch? (y/N): n ... Would you like to update an associated JIRA? (y/N): y ... Check if the JIRA information is as expected (Y/n): y # <-- Inconsistent. ``` ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45427 from HyukjinKwon/minor-script. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- dev/merge_spark_pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 4565d055e400e..c9893fd7e5a9d 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -283,7 +283,7 @@ def get_jira_issue(prompt, default_jira_id=""): if status == "Resolved" or status == "Closed": print("JIRA issue %s already has status '%s'" % (jira_id, status)) return None - if bold_input("Check if the JIRA information is as expected (Y/n): ").lower() != "n": + if bold_input("Check if the JIRA information is as expected (y/N): ").lower() == "y": return issue else: return get_jira_issue("Enter the revised JIRA ID again or leave blank to skip") From 54f15721649d8cb094043b8b6f45985a8005ad55 Mon Sep 17 00:00:00 2001 From: zwangsheng Date: Fri, 8 Mar 2024 10:04:38 +0800 Subject: [PATCH 05/50] [SPARK-47314][DOC] Remove the wrong comment line of `ExternalSorter#writePartitionedMapOutput` method ### What changes were proposed in this pull request? Correct the comment of `ExternalSorter#writePartitionedMapOutput`. `ExternalSorter#writePartitionedMapOutput` return nothing, and will update the partition's length when call partition pair writer to close. ### Why are the changes needed? Correct comment. ### Does this PR introduce _any_ user-facing change? No, developers will meet this change in source code. ### How was this patch tested? No need. ### Was this patch authored or co-authored using generative AI tooling? No Closes #45415 from zwangsheng/SPARK-47314. Authored-by: zwangsheng Signed-off-by: Kent Yao --- .../scala/org/apache/spark/util/collection/ExternalSorter.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 9db7fd18b07ab..77aec10a6b126 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -689,8 +689,6 @@ private[spark] class ExternalSorter[K, V, C]( /** * Write all the data added into this ExternalSorter into a map output writer that pushes bytes * to some arbitrary backing store. This is called by the SortShuffleWriter. - * - * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) */ def writePartitionedMapOutput( shuffleId: Int, From f659f8d1b019385ad95673205386b6cbe8f89a49 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Fri, 8 Mar 2024 13:15:35 +0800 Subject: [PATCH 06/50] [SPARK-47319][SQL] Improve missingInput calculation ### What changes were proposed in this pull request? This PR improves `QueryPlan.missingInput()` calculation. ### Why are the changes needed? This seems to be the root cause of `DeduplicateRelations` slowness in some cases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing UTs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45424 from peter-toth/fix-missinginput. Authored-by: Peter Toth Signed-off-by: Kent Yao --- .../catalyst/expressions/AttributeSet.scala | 20 ++++++++++++------- .../spark/sql/catalyst/plans/QueryPlan.scala | 8 +++++++- 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 2628afd8923c2..236380b2c030b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -104,13 +104,19 @@ class AttributeSet private (private val baseSet: mutable.LinkedHashSet[Attribute * in `other`. */ def --(other: Iterable[NamedExpression]): AttributeSet = { - other match { - // SPARK-32755: `--` method behave differently under scala 2.12 and 2.13, - // use a Scala 2.12 based code to maintains the insertion order in Scala 2.13 - case otherSet: AttributeSet => - new AttributeSet(baseSet.clone() --= otherSet.baseSet) - case _ => - new AttributeSet(baseSet.clone() --= other.map(a => new AttributeEquals(a.toAttribute))) + if (isEmpty) { + AttributeSet.empty + } else if (other.isEmpty) { + this + } else { + other match { + // SPARK-32755: `--` method behave differently under scala 2.12 and 2.13, + // use a Scala 2.12 based code to maintains the insertion order in Scala 2.13 + case otherSet: AttributeSet => + new AttributeSet(baseSet.clone() --= otherSet.baseSet) + case _ => + new AttributeSet(baseSet.clone() --= other.map(a => new AttributeEquals(a.toAttribute))) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 2a62ea1feb031..0f049103542ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -102,7 +102,13 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] /** * Attributes that are referenced by expressions but not provided by this node's children. */ - final def missingInput: AttributeSet = references -- inputSet + final def missingInput: AttributeSet = { + if (references.isEmpty) { + AttributeSet.empty + } else { + references -- inputSet + } + } /** * Runs [[transformExpressionsDown]] with `rule` on all expressions present From c0a8cf9cca08b278b41c2149d8067debe98c56e9 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 8 Mar 2024 16:13:02 +0900 Subject: [PATCH 07/50] [SPARK-47079][PYTHON][DOCS][FOLLOWUP] Add `VariantType` to API references ### What changes were proposed in this pull request? Add `VariantType` to API references ### Why are the changes needed? `VariantType` has been added in `__all__` in `types` ### Does this PR introduce _any_ user-facing change? yes ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #45429 from zhengruifeng/doc_variant_type. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/docs/source/reference/pyspark.sql/data_types.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/python/docs/source/reference/pyspark.sql/data_types.rst b/python/docs/source/reference/pyspark.sql/data_types.rst index 65516b7b3e166..d201be271a6a6 100644 --- a/python/docs/source/reference/pyspark.sql/data_types.rst +++ b/python/docs/source/reference/pyspark.sql/data_types.rst @@ -44,6 +44,7 @@ Data Types VarcharType StructField StructType + VariantType TimestampType TimestampNTZType DayTimeIntervalType From 50131054b818235b2f961f98e44c5fb859a1489f Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 8 Mar 2024 16:49:29 +0800 Subject: [PATCH 08/50] [SPARK-47265][SQL][TESTS][COLUMN] Replace `createTable(..., schema: StructType, ...)` with `createTable(..., columns: Array, ...)` in UT ### What changes were proposed in this pull request? The pr aims to use `createTable(..., schema: StructType, ...)` instead of `createTable(..., columns: Array[Column], ...)` in UT. ### Why are the changes needed? Because the `TableCatalog#createTable(..., schema: StructType, ...)` method has been marked as `deprecated` after version `3.4.0`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Update existed UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45368 from panbingkun/SPARK-47265. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../connect/ProtoToParsedPlanTestSuite.scala | 8 +- .../sql/jdbc/v2/V2JDBCNamespaceTest.scala | 12 +- .../catalog/StagingTableCatalog.java | 2 +- .../sql/errors/QueryCompilationErrors.scala | 10 + .../sql/connector/catalog/CatalogSuite.scala | 212 +++++++++--------- .../InMemoryPartitionTableCatalog.scala | 4 +- ...nMemoryRowLevelOperationTableCatalog.scala | 4 +- .../catalog/InMemoryTableCatalog.scala | 12 +- .../InMemoryTableWithV2FilterCatalog.scala | 4 +- ...pportsAtomicPartitionManagementSuite.scala | 10 +- .../SupportsPartitionManagementSuite.scala | 18 +- .../datasources/v2/V2SessionCatalog.scala | 18 +- .../v2/jdbc/JDBCTableCatalog.scala | 11 +- .../DataSourceV2DataFrameSuite.scala | 6 +- .../KeyGroupedPartitioningSuite.scala | 173 +++++++------- .../spark/sql/connector/LocalScanSuite.scala | 4 +- .../connector/TestV2SessionCatalogBase.scala | 12 +- .../sql/connector/V1ReadFallbackSuite.scala | 5 +- .../WriteDistributionAndOrderingSuite.scala | 23 +- .../InMemoryTableMetricSuite.scala | 6 +- .../v2/V2SessionCatalogSuite.scala | 210 ++++++++--------- .../test/DataStreamTableAPISuite.scala | 4 +- 22 files changed, 391 insertions(+), 377 deletions(-) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala index e0c4e21503e91..9d16efd3bea87 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala @@ -35,10 +35,10 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.connect.config.Connect import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.connect.service.SessionHolder -import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, InMemoryCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, Column, Identifier, InMemoryCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.LongType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -137,12 +137,12 @@ class ProtoToParsedPlanTestSuite inMemoryCatalog.createNamespace(Array("tempdb"), emptyProps) inMemoryCatalog.createTable( Identifier.of(Array("tempdb"), "myTable"), - new StructType().add("id", "long"), + Array(Column.create("id", LongType)), Array.empty[Transform], emptyProps) inMemoryCatalog.createTable( Identifier.of(Array("tempdb"), "myStreamingTable"), - new StructType().add("id", "long"), + Array(Column.create("id", LongType)), Array.empty[Transform], emptyProps) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala index 4eacfbfbd8804..e4cc88cec0f5e 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCNamespaceTest.scala @@ -26,12 +26,12 @@ import org.apache.logging.log4j.Level import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException -import org.apache.spark.sql.connector.catalog.{Identifier, NamespaceChange} +import org.apache.spark.sql.connector.catalog.{Column, Identifier, NamespaceChange} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.jdbc.DockerIntegrationFunSuite import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType} import org.apache.spark.tags.DockerTest @DockerTest @@ -39,9 +39,9 @@ private[v2] trait V2JDBCNamespaceTest extends SharedSparkSession with DockerInte val catalog = new JDBCTableCatalog() private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] - private val schema: StructType = new StructType() - .add("id", IntegerType) - .add("data", StringType) + private val columns: Array[Column] = Array( + Column.create("id", IntegerType), + Column.create("data", StringType)) def builtinNamespaces: Array[Array[String]] @@ -116,7 +116,7 @@ private[v2] trait V2JDBCNamespaceTest extends SharedSparkSession with DockerInte // Drop non empty namespace without cascade catalog.createNamespace(Array("foo"), commentMap.asJava) assert(catalog.namespaceExists(Array("foo")) === true) - catalog.createTable(ident1, schema, Array.empty[Transform], emptyProps) + catalog.createTable(ident1, columns, Array.empty[Transform], emptyProps) if (supportsDropSchemaRestrict) { intercept[NonEmptyNamespaceException] { catalog.dropNamespace(Array("foo"), cascade = false) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java index e69fc3324c691..a8e1757a492d8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java @@ -38,7 +38,7 @@ * SELECT and REPLACE TABLE AS SELECT operations are atomic. For example, when one runs a REPLACE * TABLE AS SELECT operation, if the catalog does not implement this trait, the planner will first * drop the table via {@link TableCatalog#dropTable(Identifier)}, then create the table via - * {@link TableCatalog#createTable(Identifier, StructType, Transform[], Map)}, and then perform + * {@link TableCatalog#createTable(Identifier, Column[], Transform[], Map)}, and then perform * the write via {@link SupportsWrite#newWriteBuilder(LogicalWriteInfo)}. * However, if the write operation fails, the catalog will have already dropped the table, and the * planner cannot roll back the dropping of the table. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index ec0202cb25442..5b304672092bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -3995,4 +3995,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat origin = e.origin ) } + + private def callDeprecatedMethodError(oldMethod: String, newMethod: String): Throwable = { + SparkException.internalError(s"The method `$oldMethod` is deprecated, " + + s"please use `$newMethod` instead.") + } + + def createTableDeprecatedError(): Throwable = { + callDeprecatedMethodError("createTable(..., StructType, ...)", + "createTable(..., Array[Column], ...)") + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala index 2935b01649cad..145bfd286123f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.functions.{BoundFunction, ScalarFunction, UnboundFunction} import org.apache.spark.sql.connector.expressions.{LogicalExpressions, Transform} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class CatalogSuite extends SparkFunSuite { @@ -38,9 +38,9 @@ class CatalogSuite extends SparkFunSuite { private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] private val emptyTrans: Array[Transform] = Array.empty - private val schema: StructType = new StructType() - .add("id", IntegerType) - .add("data", StringType) + private val columns: Array[Column] = Array( + Column.create("id", IntegerType), + Column.create("data", StringType)) private def newCatalog(): InMemoryCatalog = { val newCatalog = new InMemoryCatalog @@ -75,13 +75,13 @@ class CatalogSuite extends SparkFunSuite { intercept[NoSuchNamespaceException](catalog.listTables(Array("ns"))) - catalog.createTable(ident1, schema, emptyTrans, emptyProps) + catalog.createTable(ident1, columns, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) intercept[NoSuchNamespaceException](catalog.listTables(Array("ns2"))) - catalog.createTable(ident3, schema, emptyTrans, emptyProps) - catalog.createTable(ident2, schema, emptyTrans, emptyProps) + catalog.createTable(ident3, columns, emptyTrans, emptyProps) + catalog.createTable(ident2, columns, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) @@ -101,11 +101,11 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("test", "`", ".", "test_table")) - assert(table.schema == schema) + assert(table.columns === columns) assert(table.properties.asScala == Map()) assert(catalog.tableExists(testIdent)) @@ -119,11 +119,11 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, emptyTrans, properties) + val table = catalog.createTable(testIdent, columns, emptyTrans, properties) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("test", "`", ".", "test_table")) - assert(table.schema == schema) + assert(table.columns === columns) assert(table.properties == properties) assert(catalog.tableExists(testIdent)) @@ -134,10 +134,10 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val exc = intercept[TableAlreadyExistsException] { - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) } checkErrorTableAlreadyExists(exc, testIdentQuoted) @@ -150,7 +150,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -162,7 +162,7 @@ class CatalogSuite extends SparkFunSuite { test("loadTable") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val loaded = catalog.loadTable(testIdent) assert(table.name == loaded.name) @@ -183,7 +183,7 @@ class CatalogSuite extends SparkFunSuite { test("invalidateTable") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) catalog.invalidateTable(testIdent) val loaded = catalog.loadTable(testIdent) @@ -204,7 +204,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add property") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(table.properties.asScala == Map()) @@ -223,7 +223,7 @@ class CatalogSuite extends SparkFunSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - val table = catalog.createTable(testIdent, schema, emptyTrans, properties) + val table = catalog.createTable(testIdent, columns, emptyTrans, properties) assert(table.properties.asScala == Map("prop-1" -> "1")) @@ -242,7 +242,7 @@ class CatalogSuite extends SparkFunSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - val table = catalog.createTable(testIdent, schema, emptyTrans, properties) + val table = catalog.createTable(testIdent, columns, emptyTrans, properties) assert(table.properties.asScala == Map("prop-1" -> "1")) @@ -258,7 +258,7 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: remove missing property") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(table.properties.asScala == Map()) @@ -274,66 +274,66 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType)) - assert(updated.schema == schema.add("ts", TimestampType)) + assert(updated.columns === columns :+ Column.create("ts", TimestampType)) } test("alterTable: add required column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType, false)) - assert(updated.schema == schema.add("ts", TimestampType, nullable = false)) + assert(updated.columns === columns :+ Column.create("ts", TimestampType, false)) } test("alterTable: add column with comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType, false, "comment text")) - val field = StructField("ts", TimestampType, nullable = false).withComment("comment text") - assert(updated.schema == schema.add(field)) + val tsColumn = Column.create("ts", TimestampType, false, "comment text", null) + assert(updated.columns === (columns :+ tsColumn)) } test("alterTable: add nested column") { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("point", "z"), DoubleType)) - val expectedSchema = schema.add("point", pointStruct.add("z", DoubleType)) + val expectedColumns = columns :+ Column.create("point", pointStruct.add("z", DoubleType)) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: add column to primitive field fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -343,15 +343,15 @@ class CatalogSuite extends SparkFunSuite { parameters = Map("name" -> "data")) // the table has not changed - assert(catalog.loadTable(testIdent).schema == schema) + assert(catalog.loadTable(testIdent).columns === columns) } test("alterTable: add field to missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -365,9 +365,9 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: update column data type") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType)) @@ -378,12 +378,12 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: update column nullability") { val catalog = newCatalog() - val originalSchema = new StructType() - .add("id", IntegerType, nullable = false) - .add("data", StringType) - val table = catalog.createTable(testIdent, originalSchema, emptyTrans, emptyProps) + val originalColumns = Array( + Column.create("id", IntegerType, false), + Column.create("data", StringType)) + val table = catalog.createTable(testIdent, originalColumns, emptyTrans, emptyProps) - assert(table.schema == originalSchema) + assert(table.columns === originalColumns) val updated = catalog.alterTable(testIdent, TableChange.updateColumnNullability(Array("id"), true)) @@ -395,9 +395,9 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: update missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -411,9 +411,9 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) @@ -427,9 +427,9 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: replace comment") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) @@ -446,9 +446,9 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: add comment to missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -462,9 +462,9 @@ class CatalogSuite extends SparkFunSuite { test("alterTable: rename top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("id"), "some_id")) @@ -477,46 +477,46 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("point", "x"), "first")) val newPointStruct = new StructType().add("first", DoubleType).add("y", DoubleType) - val expectedSchema = schema.add("point", newPointStruct) + val expectedColumns = columns :+ Column.create("point", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: rename struct column") { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("point"), "p")) val newPointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val expectedSchema = schema.add("p", newPointStruct) + val expectedColumns = columns :+ Column.create("p", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: rename missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -531,28 +531,28 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("point", "x"), "first"), TableChange.renameColumn(Array("point", "y"), "second")) val newPointStruct = new StructType().add("first", DoubleType).add("second", DoubleType) - val expectedSchema = schema.add("point", newPointStruct) + val expectedColumns = columns :+ Column.create("point", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns() === expectedColumns) } test("alterTable: delete top-level column") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) val updated = catalog.alterTable(testIdent, TableChange.deleteColumn(Array("id"), false)) @@ -565,27 +565,27 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) val updated = catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "y"), false)) val newPointStruct = new StructType().add("x", DoubleType) - val expectedSchema = schema.add("point", newPointStruct) + val expectedColumns = columns :+ Column.create("point", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: delete missing column fails") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) - assert(table.schema == schema) + assert(table.columns === columns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -596,18 +596,18 @@ class CatalogSuite extends SparkFunSuite { // with if exists it should pass catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), true)) - assert(table.schema == schema) + assert(table.columns === columns) } test("alterTable: delete missing nested column fails") { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - val table = catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) checkError( exception = intercept[SparkIllegalArgumentException] { @@ -618,7 +618,7 @@ class CatalogSuite extends SparkFunSuite { // with if exists it should pass catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), true)) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) } test("alterTable: table does not exist") { @@ -636,7 +636,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -668,7 +668,7 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) catalog.renameTable(testIdent, testIdentNew) @@ -693,8 +693,8 @@ class CatalogSuite extends SparkFunSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) - catalog.createTable(testIdentNew, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) + catalog.createTable(testIdentNew, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) assert(catalog.tableExists(testIdentNew)) @@ -720,8 +720,8 @@ class CatalogSuite extends SparkFunSuite { val ident1 = Identifier.of(Array("ns1", "ns2"), "test_table_1") val ident2 = Identifier.of(Array("ns1", "ns2"), "test_table_2") - catalog.createTable(ident1, schema, emptyTrans, emptyProps) - catalog.createTable(ident2, schema, emptyTrans, emptyProps) + catalog.createTable(ident1, columns, emptyTrans, emptyProps) + catalog.createTable(ident2, columns, emptyTrans, emptyProps) assert(catalog.listNamespaces === Array(Array("ns1"))) assert(catalog.listNamespaces(Array()) === Array(Array("ns1"))) @@ -735,8 +735,8 @@ class CatalogSuite extends SparkFunSuite { val ident2 = Identifier.of(Array("ns1", "ns2"), "test_table_2") catalog.createNamespace(Array("ns1"), Map("property" -> "value").asJava) - catalog.createTable(ident1, schema, emptyTrans, emptyProps) - catalog.createTable(ident2, schema, emptyTrans, emptyProps) + catalog.createTable(ident1, columns, emptyTrans, emptyProps) + catalog.createTable(ident2, columns, emptyTrans, emptyProps) assert(catalog.listNamespaces === Array(Array("ns1"))) assert(catalog.listNamespaces(Array()) === Array(Array("ns1"))) @@ -757,7 +757,7 @@ class CatalogSuite extends SparkFunSuite { test("loadNamespaceMetadata: no metadata, table exists") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val metadata = catalog.loadNamespaceMetadata(testNs) @@ -778,7 +778,7 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val metadata = catalog.loadNamespaceMetadata(testNs) @@ -811,7 +811,7 @@ class CatalogSuite extends SparkFunSuite { test("createNamespace: fail if namespace already exists from table") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.namespaceExists(testNs) === true) assert(catalog.loadNamespaceMetadata(testNs).asScala === Map.empty) @@ -853,7 +853,7 @@ class CatalogSuite extends SparkFunSuite { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.dropNamespace(testNs, cascade = true)) @@ -883,7 +883,7 @@ class CatalogSuite extends SparkFunSuite { test("alterNamespace: create metadata if missing and table exists") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) catalog.alterNamespace(testNs, NamespaceChange.setProperty("property", "value")) @@ -903,7 +903,7 @@ class CatalogSuite extends SparkFunSuite { test("truncate non-partitioned table") { val catalog = newCatalog() - val table = catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + val table = catalog.createTable(testIdent, columns, emptyTrans, emptyProps) .asInstanceOf[InMemoryTable] table.withData(Array( new BufferedRows("3").withRow(InternalRow(0, "abc", "3")), @@ -918,9 +918,9 @@ class CatalogSuite extends SparkFunSuite { val table = partCatalog.createTable( testIdent, - new StructType() - .add("col0", IntegerType) - .add("part0", IntegerType), + Array( + Column.create("col0", IntegerType), + Column.create("part0", IntegerType)), Array[Transform](LogicalExpressions.identity(LogicalExpressions.parseReference("part0"))), util.Collections.emptyMap[String, String]) val partTable = table.asInstanceOf[InMemoryPartitionTable] @@ -933,7 +933,7 @@ class CatalogSuite extends SparkFunSuite { new BufferedRows("1").withRow(InternalRow(1, 1)) )) assert(partTable.listPartitionIdentifiers(Array.empty, InternalRow.empty).length == 2) - assert(!partTable.rows.isEmpty) + assert(partTable.rows.nonEmpty) assert(partTable.truncateTable()) assert(partTable.listPartitionIdentifiers(Array.empty, InternalRow.empty).length == 2) assert(partTable.rows.isEmpty) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala index 9a45d64209837..3b8020003aa4a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryPartitionTableCatalog.scala @@ -21,14 +21,13 @@ import java.util import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.StructType class InMemoryPartitionTableCatalog extends InMemoryTableCatalog { import CatalogV2Implicits._ override def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { @@ -37,6 +36,7 @@ class InMemoryPartitionTableCatalog extends InMemoryTableCatalog { InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) + val schema = CatalogV2Util.v2ColumnsToStructType(columns) val table = new InMemoryAtomicPartitionTable( s"$name.${ident.quoted}", schema, partitions, properties) tables.put(ident, table) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala index 94e6947612219..deb200650bd52 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTableCatalog.scala @@ -21,14 +21,13 @@ import java.util import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.StructType class InMemoryRowLevelOperationTableCatalog extends InMemoryTableCatalog { import CatalogV2Implicits._ override def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { @@ -38,6 +37,7 @@ class InMemoryRowLevelOperationTableCatalog extends InMemoryTableCatalog { InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) val tableName = s"$name.${ident.quoted}" + val schema = CatalogV2Util.v2ColumnsToStructType(columns) val table = new InMemoryRowLevelOperationTable(tableName, schema, partitions, properties) tables.put(ident, table) namespaces.putIfAbsent(ident.namespace.toList, Map()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index f1f84fbfeb9f9..d511477ef5d33 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -25,6 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NonEmptyNamespaceException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.distributions.{Distribution, Distributions} import org.apache.spark.sql.connector.expressions.{SortOrder, Transform} +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -84,14 +85,12 @@ class BasicInMemoryTableCatalog extends TableCatalog { invalidatedTables.add(ident) } - // TODO: remove it when no tests calling this deprecated method. override def createTable( ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): Table = { - createTable(ident, schema, partitions, properties, Distributions.unspecified(), - Array.empty, None, None) + throw QueryCompilationErrors.createTableDeprecatedError() } override def createTable( @@ -99,13 +98,13 @@ class BasicInMemoryTableCatalog extends TableCatalog { columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String]): Table = { - val schema = CatalogV2Util.v2ColumnsToStructType(columns) - createTable(ident, schema, partitions, properties) + createTable(ident, columns, partitions, properties, Distributions.unspecified(), + Array.empty, None, None) } def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String], distribution: Distribution, @@ -114,6 +113,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { advisoryPartitionSize: Option[Long], distributionStrictlyRequired: Boolean = true, numRowsPerSplit: Int = Int.MaxValue): Table = { + val schema = CatalogV2Util.v2ColumnsToStructType(columns) if (tables.containsKey(ident)) { throw new TableAlreadyExistsException(ident.asMultipartIdentifier) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala index 240550fdcf99d..7ec1cab304ade 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableWithV2FilterCatalog.scala @@ -21,14 +21,13 @@ import java.util import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.StructType class InMemoryTableWithV2FilterCatalog extends InMemoryTableCatalog { import CatalogV2Implicits._ override def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { @@ -38,6 +37,7 @@ class InMemoryTableWithV2FilterCatalog extends InMemoryTableCatalog { InMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) val tableName = s"$name.${ident.quoted}" + val schema = CatalogV2Util.v2ColumnsToStructType(columns) val table = new InMemoryTableWithV2Filter(tableName, schema, partitions, properties) tables.put(ident, table) namespaces.putIfAbsent(ident.namespace.toList, Map()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala index 4d25fda92ec1e..1aa0b408366bf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsAtomicPartitionManagementSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.{SparkFunSuite, SparkUnsupportedOperationException} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class SupportsAtomicPartitionManagementSuite extends SparkFunSuite { @@ -37,10 +37,10 @@ class SupportsAtomicPartitionManagementSuite extends SparkFunSuite { newCatalog.initialize("test", CaseInsensitiveStringMap.empty()) newCatalog.createTable( ident, - new StructType() - .add("id", IntegerType) - .add("data", StringType) - .add("dt", StringType), + Array( + Column.create("id", IntegerType), + Column.create("data", StringType), + Column.create("dt", StringType)), Array[Transform](LogicalExpressions.identity(ref("dt"))), util.Collections.emptyMap[String, String]) newCatalog diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala index 698ac5b0ba40a..06a23e7fda207 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/SupportsPartitionManagementSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException, SparkUnsu import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, PartitionsAlreadyExistException} import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} -import org.apache.spark.sql.types.{IntegerType, StringType, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class SupportsPartitionManagementSuite extends SparkFunSuite { @@ -39,10 +39,10 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { newCatalog.initialize("test", CaseInsensitiveStringMap.empty()) newCatalog.createTable( ident, - new StructType() - .add("id", IntegerType) - .add("data", StringType) - .add("dt", StringType), + Array( + Column.create("id", IntegerType), + Column.create("data", StringType), + Column.create("dt", StringType)), Array[Transform](LogicalExpressions.identity(ref("dt"))), util.Collections.emptyMap[String, String]) newCatalog @@ -163,10 +163,10 @@ class SupportsPartitionManagementSuite extends SparkFunSuite { partCatalog.initialize("test", CaseInsensitiveStringMap.empty()) val table = partCatalog.createTable( ident, - new StructType() - .add("col0", IntegerType) - .add("part0", IntegerType) - .add("part1", StringType), + Array( + Column.create("col0", IntegerType), + Column.create("part0", IntegerType), + Column.create("part1", StringType)), Array[Transform]( LogicalExpressions.identity(ref("part0")), LogicalExpressions.identity(ref("part1"))), util.Collections.emptyMap[String, String]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index ebb7c307b926a..15166c8229034 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -156,15 +156,6 @@ class V2SessionCatalog(catalog: SessionCatalog) columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String]): Table = { - createTable(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties) - } - - // TODO: remove it when no tests calling this deprecated method. - override def createTable( - ident: Identifier, - schema: StructType, - partitions: Array[Transform], - properties: util.Map[String, String]): Table = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val provider = properties.getOrDefault(TableCatalog.PROP_PROVIDER, conf.defaultDataSourceName) val tableProperties = properties.asScala.toMap @@ -178,6 +169,7 @@ class V2SessionCatalog(catalog: SessionCatalog) CatalogTableType.MANAGED } + val schema = CatalogV2Util.v2ColumnsToStructType(columns) val (newSchema, newPartitions) = DataSourceV2Utils.getTableProvider(provider, conf) match { // If the provider does not support external metadata, users should not be allowed to // specify custom schema when creating the data source table, since the schema will not @@ -249,6 +241,14 @@ class V2SessionCatalog(catalog: SessionCatalog) null // Return null to save the `loadTable` call for CREATE TABLE without AS SELECT. } + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + throw QueryCompilationErrors.createTableDeprecatedError() + } + private def toOptions(properties: Map[String, String]): Map[String, String] = { properties.filter { case (k, _) => k.startsWith(TableCatalog.OPTION_PREFIX) }.map { case (key, value) => key.drop(TableCatalog.OPTION_PREFIX.length) -> value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala index a5ef9bd5e6b89..b6e3619464776 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException -import org.apache.spark.sql.connector.catalog.{FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors, QueryExecutionErrors} @@ -144,6 +144,14 @@ class JDBCTableCatalog extends TableCatalog schema: StructType, partitions: Array[Transform], properties: java.util.Map[String, String]): Table = { + throw QueryCompilationErrors.createTableDeprecatedError() + } + + override def createTable( + ident: Identifier, + columns: Array[Column], + partitions: Array[Transform], + properties: java.util.Map[String, String]): Table = { checkNamespace(ident.namespace()) if (partitions.nonEmpty) { throw QueryExecutionErrors.cannotCreateJDBCTableWithPartitionsError() @@ -180,6 +188,7 @@ class JDBCTableCatalog extends TableCatalog val writeOptions = new JdbcOptionsInWrite(tableOptions) val caseSensitive = SQLConf.get.caseSensitiveAnalysis + val schema = CatalogV2Util.v2ColumnsToStructType(columns) JdbcUtils.withConnection(options) { conn => JdbcUtils.classifyException( errorClass = "FAILED_JDBC.CREATE_TABLE", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index b8e0d50dc9c00..7d48459a8a517 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -22,12 +22,12 @@ import java.util.Collections import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, ReplaceTableAsSelect} -import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.CalendarIntervalType import org.apache.spark.sql.util.QueryExecutionListener class DataSourceV2DataFrameSuite @@ -178,7 +178,7 @@ class DataSourceV2DataFrameSuite val testCatalog = spark.sessionState.catalogManager.catalog("testcat").asTableCatalog testCatalog.createTable( Identifier.of(Array(), "table_name"), - new StructType().add("i", "interval"), + Array(Column.create("i", CalendarIntervalType)), Array.empty[Transform], Collections.emptyMap[String, String]) val df = sql(s"select interval 1 millisecond as i") val v2Writer = df.writeTo("testcat.table_name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala index e6448d4d80fda..7fdc703007c2c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -23,8 +23,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Literal, TransformExpression} import org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.connector.catalog.Identifier -import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog +import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog} import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.connector.distributions.Distributions import org.apache.spark.sql.connector.expressions._ @@ -64,16 +63,16 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { Collections.emptyMap[String, String] } private val table: String = "tbl" - private val schema = new StructType() - .add("id", IntegerType) - .add("data", StringType) - .add("ts", TimestampType) + private val columns: Array[Column] = Array( + Column.create("id", IntegerType), + Column.create("data", StringType), + Column.create("ts", TimestampType)) test("clustered distribution: output partitioning should be KeyGroupedPartitioning") { val partitions: Array[Transform] = Array(Expressions.years("ts")) // create a table with 3 partitions, partitioned by `years` transform - createTable(table, schema, partitions) + createTable(table, columns, partitions) sql(s"INSERT INTO testcat.ns.$table VALUES " + s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " + s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " + @@ -98,7 +97,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("non-clustered distribution: no partition") { val partitions: Array[Transform] = Array(bucket(32, "ts")) - createTable(table, schema, partitions) + createTable(table, columns, partitions) val df = sql(s"SELECT * FROM testcat.ns.$table") val distribution = physical.ClusteredDistribution( @@ -109,7 +108,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("non-clustered distribution: single partition") { val partitions: Array[Transform] = Array(bucket(32, "ts")) - createTable(table, schema, partitions) + createTable(table, columns, partitions) sql(s"INSERT INTO testcat.ns.$table VALUES (0, 'aaa', CAST('2020-01-01' AS timestamp))") val df = sql(s"SELECT * FROM testcat.ns.$table") @@ -127,7 +126,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { val nonFunctionCatalog = spark.sessionState.catalogManager.catalog("testcat2") .asInstanceOf[InMemoryTableCatalog] val partitions: Array[Transform] = Array(bucket(32, "ts")) - createTable(table, schema, partitions, catalog = nonFunctionCatalog) + createTable(table, columns, partitions, catalog = nonFunctionCatalog) sql(s"INSERT INTO testcat2.ns.$table VALUES " + s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " + s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " + @@ -147,7 +146,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { catalog.clearFunctions() val partitions: Array[Transform] = Array(bucket(32, "ts")) - createTable(table, schema, partitions) + createTable(table, columns, partitions) sql(s"INSERT INTO testcat.ns.$table VALUES " + s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " + s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " + @@ -162,7 +161,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("non-clustered distribution: V2 bucketing disabled") { withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "false") { val partitions: Array[Transform] = Array(bucket(32, "ts")) - createTable(table, schema, partitions) + createTable(table, columns, partitions) sql(s"INSERT INTO testcat.ns.$table VALUES " + s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " + s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " + @@ -182,7 +181,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { ) // create a table with 3 partitions, partitioned by `truncate` transform - createTable(table, schema, partitions) + createTable(table, columns, partitions) sql(s"INSERT INTO testcat.ns.$table VALUES " + s"(0, 'aaa', CAST('2022-01-01' AS timestamp)), " + s"(1, 'bbb', CAST('2021-01-01' AS timestamp)), " + @@ -227,24 +226,24 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { private def createTable( table: String, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], catalog: InMemoryTableCatalog = catalog): Unit = { catalog.createTable(Identifier.of(Array("ns"), table), - schema, partitions, emptyProps, Distributions.unspecified(), Array.empty, None, None, + columns, partitions, emptyProps, Distributions.unspecified(), Array.empty, None, None, numRowsPerSplit = 1) } private val customers: String = "customers" - private val customers_schema = new StructType() - .add("customer_name", StringType) - .add("customer_age", IntegerType) - .add("customer_id", LongType) + private val customersColumns: Array[Column] = Array( + Column.create("customer_name", StringType), + Column.create("customer_age", IntegerType), + Column.create("customer_id", LongType)) private val orders: String = "orders" - private val orders_schema = new StructType() - .add("order_amount", DoubleType) - .add("customer_id", LongType) + private val ordersColumns: Array[Column] = Array( + Column.create("order_amount", DoubleType), + Column.create("customer_id", LongType)) private def selectWithMergeJoinHint(t1: String, t2: String): String = { s"SELECT /*+ MERGE($t1, $t2) */ " @@ -269,11 +268,11 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { customers_partitions: Array[Transform], orders_partitions: Array[Transform], expectedNumOfShuffleExecs: Int): Unit = { - createTable(customers, customers_schema, customers_partitions) + createTable(customers, customersColumns, customers_partitions) sql(s"INSERT INTO testcat.ns.$customers VALUES " + s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)") - createTable(orders, orders_schema, orders_partitions) + createTable(orders, ordersColumns, orders_partitions) sql(s"INSERT INTO testcat.ns.$orders VALUES " + s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)") @@ -329,21 +328,21 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { } private val items: String = "items" - private val items_schema: StructType = new StructType() - .add("id", LongType) - .add("name", StringType) - .add("price", FloatType) - .add("arrive_time", TimestampType) + private val itemsColumns: Array[Column] = Array( + Column.create("id", LongType), + Column.create("name", StringType), + Column.create("price", FloatType), + Column.create("arrive_time", TimestampType)) private val purchases: String = "purchases" - private val purchases_schema: StructType = new StructType() - .add("item_id", LongType) - .add("price", FloatType) - .add("time", TimestampType) + private val purchasesColumns: Array[Column] = Array( + Column.create("item_id", LongType), + Column.create("price", FloatType), + Column.create("time", TimestampType)) test("partitioned join: join with two partition keys and matching & sorted partitions") { val items_partitions = Array(bucket(8, "id"), days("arrive_time")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + @@ -352,7 +351,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(bucket(8, "item_id"), days("time")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + @@ -375,7 +374,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("partitioned join: join with two partition keys and unsorted partitions") { val items_partitions = Array(bucket(8, "id"), days("arrive_time")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + @@ -384,7 +383,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp))") val purchases_partitions = Array(bucket(8, "item_id"), days("time")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + @@ -407,7 +406,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("partitioned join: join with two partition keys and different # of partition keys") { val items_partitions = Array(bucket(8, "id"), days("arrive_time")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + @@ -415,7 +414,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(bucket(8, "item_id"), days("time")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + s"(2, 11.0, cast('2020-01-01' as timestamp))") @@ -440,7 +439,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41413: partitioned join: partition values from one side are subset of those from " + "the other side") { val items_partitions = Array(bucket(4, "id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + @@ -448,7 +447,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(bucket(4, "item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + @@ -472,7 +471,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41413: partitioned join: partition values from both sides overlaps") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + @@ -480,7 +479,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(2, 19.5, cast('2020-02-01' as timestamp)), " + @@ -504,14 +503,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41413: partitioned join: non-overlapping partition values from both sides") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(4, 42.0, cast('2020-01-01' as timestamp)), " + "(5, 19.5, cast('2020-02-01' as timestamp)), " + @@ -535,14 +534,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: with same partition keys and one side fully clustered") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + @@ -573,7 +572,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: with same partition keys and both sides partially " + "clustered") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -581,7 +580,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + @@ -617,7 +616,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: with different partition keys and both sides partially " + "clustered") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -626,7 +625,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + @@ -667,7 +666,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: with different partition keys and missing keys on " + "left-hand side") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -675,7 +674,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + @@ -714,7 +713,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: with different partition keys and missing keys on " + "right-hand side") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -722,7 +721,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + @@ -755,7 +754,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: left outer join") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -764,7 +763,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(2, 20.0, cast('2020-01-01' as timestamp)), " + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + @@ -802,7 +801,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: right outer join") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -810,7 +809,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + s"(2, 15.0, cast('2020-01-01' as timestamp)), " + @@ -853,7 +852,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: full outer join is not applicable") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + @@ -861,7 +860,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-01-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + s"(2, 15.0, cast('2020-01-01' as timestamp)), " + @@ -907,7 +906,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "10") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + @@ -916,7 +915,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + @@ -946,7 +945,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-42038: partially clustered: with dynamic partition filtering") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + @@ -956,7 +955,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + @@ -1008,14 +1007,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41471: shuffle one side: only one side reports partitioning") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") - createTable(purchases, purchases_schema, Array.empty) + createTable(purchases, purchasesColumns, Array.empty) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(3, 19.5, cast('2020-02-01' as timestamp))") @@ -1038,14 +1037,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41471: shuffle one side: shuffle side has more partition value") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") - createTable(purchases, purchases_schema, Array.empty) + createTable(purchases, purchasesColumns, Array.empty) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(3, 19.5, cast('2020-02-01' as timestamp)), " + @@ -1084,14 +1083,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41471: shuffle one side: only one side reports partitioning with two identity") { val items_partitions = Array(identity("id"), identity("arrive_time")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") - createTable(purchases, purchases_schema, Array.empty) + createTable(purchases, purchasesColumns, Array.empty) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(3, 19.5, cast('2020-02-01' as timestamp))") @@ -1114,14 +1113,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41471: shuffle one side: partitioning with transform") { val items_partitions = Array(years("arrive_time")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(4, 'cc', 15.5, cast('2021-02-01' as timestamp))") - createTable(purchases, purchases_schema, Array.empty) + createTable(purchases, purchasesColumns, Array.empty) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(3, 19.5, cast('2021-02-01' as timestamp))") @@ -1147,14 +1146,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-41471: shuffle one side: work with group partition split") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") - createTable(purchases, purchases_schema, Array.empty) + createTable(purchases, purchasesColumns, Array.empty) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(3, 19.5, cast('2020-02-01' as timestamp)), " + @@ -1174,7 +1173,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-44641: duplicated records when SPJ is not triggered") { val items_partitions = Array(bucket(8, "id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s""" INSERT INTO testcat.ns.$items VALUES (1, 'aa', 40.0, cast('2020-01-01' as timestamp)), @@ -1184,7 +1183,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { (3, 'cc', 15.5, cast('2020-02-01' as timestamp))""") val purchases_partitions = Array(bucket(8, "item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"""INSERT INTO testcat.ns.$purchases VALUES (1, 42.0, cast('2020-01-01' as timestamp)), (1, 44.0, cast('2020-01-15' as timestamp)), @@ -1227,7 +1226,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { val table1 = "tab1e1" val table2 = "table2" val partition = Array(identity("id"), identity("data")) - createTable(table1, schema, partition) + createTable(table1, columns, partition) sql(s"INSERT INTO testcat.ns.$table1 VALUES " + "(1, 'aa', cast('2020-01-01' as timestamp)), " + "(2, 'bb', cast('2020-01-01' as timestamp)), " + @@ -1237,7 +1236,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { "(3, 'ee', cast('2020-01-01' as timestamp)), " + "(3, 'ee', cast('2020-01-01' as timestamp))") - createTable(table2, schema, partition) + createTable(table2, columns, partition) sql(s"INSERT INTO testcat.ns.$table2 VALUES " + "(4, 'zz', cast('2020-01-01' as timestamp)), " + "(4, 'zz', cast('2020-01-01' as timestamp)), " + @@ -1314,13 +1313,13 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { val table1 = "tab1e1" val table2 = "table2" val partition = Array(identity("id"), identity("data")) - createTable(table1, schema, partition) + createTable(table1, columns, partition) sql(s"INSERT INTO testcat.ns.$table1 VALUES " + "(1, 'aa', cast('2020-01-01' as timestamp)), " + "(2, 'bb', cast('2020-01-02' as timestamp)), " + "(3, 'cc', cast('2020-01-03' as timestamp))") - createTable(table2, schema, partition) + createTable(table2, columns, partition) sql(s"INSERT INTO testcat.ns.$table2 VALUES " + "(4, 'aa', cast('2020-01-01' as timestamp)), " + "(5, 'bb', cast('2020-01-02' as timestamp)), " + @@ -1371,7 +1370,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-44647: test join key is the second partition key and a transform") { val items_partitions = Array(bucket(8, "id"), days("arrive_time")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + @@ -1380,7 +1379,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(bucket(8, "item_id"), days("time")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + @@ -1441,7 +1440,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { test("SPARK-44647: shuffle one side and join keys are less than partition keys") { val items_partitions = Array(identity("id"), identity("name")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + @@ -1449,7 +1448,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") - createTable(purchases, purchases_schema, Array.empty) + createTable(purchases, purchasesColumns, Array.empty) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + "(1, 42.0, cast('2020-01-01' as timestamp)), " + "(1, 89.0, cast('2020-01-03' as timestamp)), " + @@ -1486,7 +1485,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "10") { val items_partitions = Array(identity("id")) - createTable(items, items_schema, items_partitions) + createTable(items, itemsColumns, items_partitions) sql(s"INSERT INTO testcat.ns.$items VALUES " + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + @@ -1495,7 +1494,7 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") val purchases_partitions = Array(identity("item_id")) - createTable(purchases, purchases_schema, purchases_partitions) + createTable(purchases, purchasesColumns, purchases_partitions) sql(s"INSERT INTO testcat.ns.$purchases VALUES " + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/LocalScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/LocalScanSuite.scala index e3d61a846fdb4..fc808c835bb9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/LocalScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/LocalScanSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, SupportsRead, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Column, Identifier, SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.read.{LocalScan, Scan, ScanBuilder} import org.apache.spark.sql.execution.LocalTableScanExec @@ -57,7 +57,7 @@ class LocalScanSuite extends QueryTest with SharedSparkSession { class TestLocalScanCatalog extends BasicInMemoryTableCatalog { override def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: java.util.Map[String, String]): Table = { val table = new TestLocalScanTable(ident.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index 719b006c14607..3b2fc0379340b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -69,15 +69,6 @@ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends Delegating columns: Array[Column], partitions: Array[Transform], properties: java.util.Map[String, String]): Table = { - createTable(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties) - } - - // TODO: remove it when no tests calling this deprecated method. - override def createTable( - ident: Identifier, - schema: StructType, - partitions: Array[Transform], - properties: java.util.Map[String, String]): Table = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper val key = TestV2SessionCatalogBase.SIMULATE_ALLOW_EXTERNAL_PROPERTY val propsWithLocation = if (properties.containsKey(key)) { @@ -93,7 +84,8 @@ private[connector] trait TestV2SessionCatalogBase[T <: Table] extends Delegating } else { properties } - super.createTable(ident, schema, partitions, propsWithLocation) + super.createTable(ident, columns, partitions, propsWithLocation) + val schema = CatalogV2Util.v2ColumnsToStructType(columns) val t = newTable(ident.quoted, schema, partitions, propsWithLocation) addTable(ident, t) t diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala index b876240ddc373..50272fac4a4a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.connector import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, QueryTest, Row, SparkSession, SQLContext} -import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, Identifier, SupportsRead, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{BasicInMemoryTableCatalog, CatalogV2Util, Column, Identifier, SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns, V1Scan} import org.apache.spark.sql.execution.RowDataSourceScanExec @@ -101,10 +101,11 @@ class V1ReadFallbackWithCatalogSuite extends V1ReadFallbackSuite { class V1ReadFallbackCatalog extends BasicInMemoryTableCatalog { override def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: java.util.Map[String, String]): Table = { // To simplify the test implementation, only support fixed schema. + val schema = CatalogV2Util.v2ColumnsToStructType(columns) if (schema != V1ReadFallbackCatalog.schema || partitions.nonEmpty) { throw SparkUnsupportedOperationException() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala index 40938eb642478..12d5f13df01c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.plans.physical.{CoalescedBoundary, CoalescedHashPartitioning, HashPartitioning, RangePartitioning, UnknownPartitioning} -import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.{Column, Identifier} import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.connector.distributions.{Distribution, Distributions} import org.apache.spark.sql.connector.expressions._ @@ -40,7 +40,7 @@ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{StreamingQueryException, Trigger} -import org.apache.spark.sql.types.{DateType, IntegerType, LongType, ObjectType, StringType, StructType, TimestampType} +import org.apache.spark.sql.types.{DateType, IntegerType, LongType, ObjectType, StringType, TimestampType} import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.tags.SlowSQLTest @@ -69,10 +69,10 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase private val ident = Identifier.of(namespace, "test_table") private val tableNameAsString = "testcat." + ident.toString private val emptyProps = Collections.emptyMap[String, String] - private val schema = new StructType() - .add("id", IntegerType) - .add("data", StringType) - .add("day", DateType) + private val columns = Array( + Column.create("id", IntegerType), + Column.create("data", StringType), + Column.create("day", DateType)) test("ordered distribution and sort with same exprs: append") { checkOrderedDistributionAndSortWithSameExprsInVariousCases("append") @@ -977,7 +977,7 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase ) val distribution = Distributions.ordered(ordering) - catalog.createTable(ident, schema, Array.empty, emptyProps, distribution, ordering, None, None) + catalog.createTable(ident, columns, Array.empty, emptyProps, distribution, ordering, None, None) withTempDir { checkpointDir => val inputData = ContinuousMemoryStream[(Long, String, Date)] @@ -1005,7 +1005,7 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase } test("continuous mode allows unspecified distribution and empty ordering") { - catalog.createTable(ident, schema, Array.empty[Transform], emptyProps) + catalog.createTable(ident, columns, Array.empty[Transform], emptyProps) withTempDir { checkpointDir => val inputData = ContinuousMemoryStream[(Long, String, Date)] @@ -1217,9 +1217,8 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase coalesce: Boolean = false): Unit = { // scalastyle:on argcount - catalog.createTable(ident, schema, Array.empty, emptyProps, tableDistribution, - tableOrdering, tableNumPartitions, tablePartitionSize, - distributionStrictlyRequired) + catalog.createTable(ident, columns, Array.empty, emptyProps, tableDistribution, + tableOrdering, tableNumPartitions, tablePartitionSize, distributionStrictlyRequired) val df = if (!dataSkewed) { spark.createDataFrame(Seq( @@ -1320,7 +1319,7 @@ class WriteDistributionAndOrderingSuite extends DistributionAndOrderingSuiteBase outputMode: String = "append", expectAnalysisException: Boolean = false): Unit = { - catalog.createTable(ident, schema, Array.empty, emptyProps, tableDistribution, + catalog.createTable(ident, columns, Array.empty, emptyProps, tableDistribution, tableOrdering, tableNumPartitions, tablePartitionSize) withTempDir { checkpointDir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala index 33e2fc46ccba4..dee8d7ac3e794 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/InMemoryTableMetricSuite.scala @@ -22,11 +22,11 @@ import org.scalatest.BeforeAndAfter import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.functions.lit import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.IntegerType class InMemoryTableMetricSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { @@ -51,7 +51,7 @@ class InMemoryTableMetricSuite testCatalog.createTable( Identifier.of(Array(), "table_name"), - new StructType().add("i", "int"), + Array(Column.create("i", IntegerType)), Array.empty[Transform], Collections.emptyMap[String, String]) func("testcat.table_name") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index 2195768e3b08c..4de74af250006 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -31,19 +31,19 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeAndAfter { val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] val emptyTrans: Array[Transform] = Array.empty - val schema: StructType = new StructType() - .add("id", IntegerType) - .add("data", StringType) + val columns: Array[Column] = Array( + Column.create("id", IntegerType), + Column.create("data", StringType)) val testNs: Array[String] = Array("db") val defaultNs: Array[String] = Array("default") @@ -98,13 +98,13 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(catalog.listTables(Array("ns")).isEmpty) - catalog.createTable(ident1, schema, emptyTrans, emptyProps) + catalog.createTable(ident1, columns, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) assert(catalog.listTables(Array("ns2")).isEmpty) - catalog.createTable(ident3, schema, emptyTrans, emptyProps) - catalog.createTable(ident2, schema, emptyTrans, emptyProps) + catalog.createTable(ident3, columns, emptyTrans, emptyProps) + catalog.createTable(ident2, columns, emptyTrans, emptyProps) assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) @@ -126,12 +126,12 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("db", "test_table")) - assert(table.schema == schema) + assert(table.columns === columns) assert(filterV2TableProperties(table.properties) == Map()) assert(catalog.tableExists(testIdent)) @@ -145,12 +145,12 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val table = catalog.loadTable(testIdent) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) assert(parsed == Seq("db", "test_table")) - assert(table.schema == schema) + assert(table.columns === columns) assert(filterV2TableProperties(table.properties).asJava == properties) assert(catalog.tableExists(testIdent)) @@ -161,14 +161,14 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) .map(part => quoteIdentifier(part)).mkString(".") val exc = intercept[TableAlreadyExistsException] { - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) } checkErrorTableAlreadyExists(exc, parsed) @@ -189,7 +189,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) // default location - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val t1 = catalog.loadTable(testIdent).asInstanceOf[V1Table] assert(t1.catalogTable.location === spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) @@ -197,21 +197,21 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { // relative path properties.put(TableCatalog.PROP_LOCATION, "relative/path") - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val t2 = catalog.loadTable(testIdent).asInstanceOf[V1Table] assert(t2.catalogTable.location === makeQualifiedPathWithWarehouse("db.db/relative/path")) catalog.dropTable(testIdent) // absolute path without scheme properties.put(TableCatalog.PROP_LOCATION, "/absolute/path") - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val t3 = catalog.loadTable(testIdent).asInstanceOf[V1Table] assert(t3.catalogTable.location.toString === "file:///absolute/path") catalog.dropTable(testIdent) // absolute path with scheme properties.put(TableCatalog.PROP_LOCATION, "file:/absolute/path") - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val t4 = catalog.loadTable(testIdent).asInstanceOf[V1Table] assert(t4.catalogTable.location.toString === "file:/absolute/path") catalog.dropTable(testIdent) @@ -222,7 +222,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -234,11 +234,11 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("loadTable") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val loaded = catalog.loadTable(testIdent) assert(loaded.name == testIdent.toString) - assert(loaded.schema == schema) + assert(loaded.columns === columns) } test("loadTable: table does not exist") { @@ -254,7 +254,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("invalidateTable") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) catalog.invalidateTable(testIdent) @@ -276,7 +276,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add property") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) assert(filterV2TableProperties(table.properties) == Map()) @@ -297,7 +297,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val table = catalog.loadTable(testIdent) assert(filterV2TableProperties(table.properties) == Map("prop-1" -> "1")) @@ -318,7 +318,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val properties = new util.HashMap[String, String]() properties.put("prop-1", "1") - catalog.createTable(testIdent, schema, emptyTrans, properties) + catalog.createTable(testIdent, columns, emptyTrans, properties) val table = catalog.loadTable(testIdent) assert(filterV2TableProperties(table.properties) == Map("prop-1" -> "1")) @@ -336,7 +336,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: remove missing property") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) assert(filterV2TableProperties(table.properties) == Map()) @@ -354,75 +354,75 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add top-level column") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType)) val updated = catalog.loadTable(testIdent) - assert(updated.schema == schema.add("ts", TimestampType)) + assert(updated.columns() === columns :+ Column.create("ts", TimestampType)) } test("alterTable: add required column") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType, false)) val updated = catalog.loadTable(testIdent) - assert(updated.schema == schema.add("ts", TimestampType, nullable = false)) + assert(updated.columns() === columns :+ Column.create("ts", TimestampType, false)) } test("alterTable: add column with comment") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType, false, "comment text")) val updated = catalog.loadTable(testIdent) - val field = StructField("ts", TimestampType, nullable = false).withComment("comment text") - assert(updated.schema == schema.add(field)) + val tsColumn = Column.create("ts", TimestampType, false, "comment text", null) + assert(updated.columns === columns :+ tsColumn) } test("alterTable: add nested column") { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) catalog.alterTable(testIdent, TableChange.addColumn(Array("point", "z"), DoubleType)) val updated = catalog.loadTable(testIdent) - val expectedSchema = schema.add("point", pointStruct.add("z", DoubleType)) + val expectedColumns = columns :+ Column.create("point", pointStruct.add("z", DoubleType)) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: add column to primitive field fails") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) @@ -432,16 +432,16 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(exc.getMessage.contains("data")) // the table has not changed - assert(catalog.loadTable(testIdent).schema == schema) + assert(catalog.loadTable(testIdent).columns === columns) } test("alterTable: add field to missing column fails") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, @@ -455,10 +455,10 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: update column data type") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType)) val updated = catalog.loadTable(testIdent) @@ -470,29 +470,33 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: update column nullability") { val catalog = newCatalog() - val originalSchema = new StructType() - .add("id", IntegerType, nullable = false) - .add("data", StringType) - catalog.createTable(testIdent, originalSchema, emptyTrans, emptyProps) + val originalColumns: Array[Column] = Array( + Column.create("id", IntegerType, false), + Column.create("data", StringType)) + catalog.createTable(testIdent, originalColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == originalSchema) + assert(table.columns === originalColumns) catalog.alterTable(testIdent, TableChange.updateColumnNullability(Array("id"), true)) val updated = catalog.loadTable(testIdent) val expectedSchema = new StructType().add("id", IntegerType).add("data", StringType) - assert(updated.schema == expectedSchema) + val expectedColumns: Array[Column] = Array( + Column.create("id", IntegerType), + Column.create("data", StringType) + ) + assert(updated.columns === expectedColumns) } test("alterTable: update missing column fails") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, @@ -506,10 +510,10 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add comment") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) @@ -524,10 +528,10 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: replace comment") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) @@ -545,10 +549,10 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: add comment to missing column fails") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, @@ -562,10 +566,10 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { test("alterTable: rename top-level column") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.renameColumn(Array("id"), "some_id")) val updated = catalog.loadTable(testIdent) @@ -579,50 +583,50 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) catalog.alterTable(testIdent, TableChange.renameColumn(Array("point", "x"), "first")) val updated = catalog.loadTable(testIdent) val newPointStruct = new StructType().add("first", DoubleType).add("y", DoubleType) - val expectedSchema = schema.add("point", newPointStruct) + val expectedColumns = columns :+ Column.create("point", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: rename struct column") { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) catalog.alterTable(testIdent, TableChange.renameColumn(Array("point"), "p")) val updated = catalog.loadTable(testIdent) val newPointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val expectedSchema = schema.add("p", newPointStruct) + val expectedColumns = columns :+ Column.create("p", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: rename missing column fails") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, @@ -637,12 +641,12 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) catalog.alterTable(testIdent, TableChange.renameColumn(Array("point", "x"), "first"), @@ -650,18 +654,18 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val updated = catalog.loadTable(testIdent) val newPointStruct = new StructType().add("first", DoubleType).add("second", DoubleType) - val expectedSchema = schema.add("point", newPointStruct) + val expectedColumns = columns :+ Column.create("point", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: delete top-level column") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) catalog.alterTable(testIdent, TableChange.deleteColumn(Array("id"), false)) val updated = catalog.loadTable(testIdent) @@ -674,29 +678,29 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "y"), false)) val updated = catalog.loadTable(testIdent) val newPointStruct = new StructType().add("x", DoubleType) - val expectedSchema = schema.add("point", newPointStruct) + val expectedColumns = columns :+ Column.create("point", newPointStruct) - assert(updated.schema == expectedSchema) + assert(updated.columns === expectedColumns) } test("alterTable: delete missing column fails") { val catalog = newCatalog() - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == schema) + assert(table.columns === columns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), false)) @@ -707,19 +711,19 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { // with if exists it should pass catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"), true)) - assert(table.schema == schema) + assert(table.columns === columns) } test("alterTable: delete missing nested column fails") { val catalog = newCatalog() val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) - val tableSchema = schema.add("point", pointStruct) + val tableColumns = columns :+ Column.create("point", pointStruct) - catalog.createTable(testIdent, tableSchema, emptyTrans, emptyProps) + catalog.createTable(testIdent, tableColumns, emptyTrans, emptyProps) val table = catalog.loadTable(testIdent) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) val exc = intercept[IllegalArgumentException] { catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), false)) @@ -730,7 +734,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { // with if exists it should pass catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"), true)) - assert(table.schema == tableSchema) + assert(table.columns === tableColumns) } test("alterTable: table does not exist") { @@ -748,7 +752,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) // default location - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val t1 = catalog.loadTable(testIdent).asInstanceOf[V1Table] assert(t1.catalogTable.location === spark.sessionState.catalog.defaultTablePath(testIdent.asTableIdentifier)) @@ -777,7 +781,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -804,7 +808,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) catalog.renameTable(testIdent, testIdentNew) @@ -829,8 +833,8 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNew)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) - catalog.createTable(testIdentNew, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) + catalog.createTable(testIdentNew, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) assert(catalog.tableExists(testIdentNew)) @@ -849,7 +853,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { assert(!catalog.tableExists(testIdent)) assert(!catalog.tableExists(testIdentNewOtherDb)) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) assert(catalog.tableExists(testIdent)) @@ -1036,7 +1040,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { assert(catalog.namespaceExists(testNs) === false) val exc = intercept[NoSuchDatabaseException] { - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) } assert(exc.getMessage.contains(testNs.quoted)) @@ -1070,7 +1074,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) - catalog.createTable(testIdent, schema, emptyTrans, emptyProps) + catalog.createTable(testIdent, columns, emptyTrans, emptyProps) val exc = intercept[AnalysisException] { catalog.dropNamespace(testNs, cascade = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index eecc9468649d2..af07aceaed145 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 import org.apache.spark.sql.connector.{FakeV2Provider, FakeV2ProviderWithCustomSchema, InMemoryTableSessionCatalog} -import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTableCatalog, MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability, V2TableWithV1Fallback} +import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog, MetadataColumn, SupportsMetadataColumns, SupportsRead, Table, TableCapability, V2TableWithV1Fallback} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.execution.streaming.{MemoryStream, MemoryStreamScanBuilder, StreamingQueryWrapper} @@ -649,7 +649,7 @@ class InMemoryStreamTableCatalog extends InMemoryTableCatalog { override def createTable( ident: Identifier, - schema: StructType, + columns: Array[Column], partitions: Array[Transform], properties: util.Map[String, String]): Table = { if (tables.containsKey(ident)) { From 00de3ed5e0aa0accc53d62b5b24fecc4e68490da Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 8 Mar 2024 17:20:48 +0800 Subject: [PATCH 09/50] [MINOR][DOCS] Fix the doc for Integral types ### What changes were proposed in this pull request? Fix the doc for Integral types ### Why are the changes needed? it seems the description of integral type are not correctly rendered before: https://spark.apache.org/docs/latest/api/python/reference/pyspark.sql/data_types.html ![image](https://github.com/apache/spark/assets/7322292/edf1061e-fea4-427d-ae72-010b180a2cd3) after: ![image](https://github.com/apache/spark/assets/7322292/cf53494e-f4b1-4c86-83b2-ecf8cad806c7) ### Does this PR introduce _any_ user-facing change? yes, doc ### How was this patch tested? manually check ### Was this patch authored or co-authored using generative AI tooling? no Closes #45434 from zhengruifeng/nit_py_datatype_doc. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/types.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index ffdd4ff576406..c4647b90767f5 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -443,21 +443,21 @@ class FloatType(FractionalType, metaclass=DataTypeSingleton): class ByteType(IntegralType): - """Byte data type, i.e. a signed integer in a single byte.""" + """Byte data type, representing signed 8-bit integers.""" def simpleString(self) -> str: return "tinyint" class IntegerType(IntegralType): - """Int data type, i.e. a signed 32-bit integer.""" + """Int data type, representing signed 32-bit integers.""" def simpleString(self) -> str: return "int" class LongType(IntegralType): - """Long data type, i.e. a signed 64-bit integer. + """Long data type, representing signed 64-bit integers. If the values are beyond the range of [-9223372036854775808, 9223372036854775807], please use :class:`DecimalType`. @@ -468,7 +468,7 @@ def simpleString(self) -> str: class ShortType(IntegralType): - """Short data type, i.e. a signed 16-bit integer.""" + """Short data type, representing signed 16-bit integers.""" def simpleString(self) -> str: return "smallint" From 7a3b3c612d16dc3655331ea49d30d4d955a6a657 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 8 Mar 2024 18:11:14 +0800 Subject: [PATCH 10/50] [SPARK-47316][SQL] Fix TimestampNTZ in Postgres Array ### What changes were proposed in this pull request? For Postgres, TimestampNTZ works well for plain TimestampNTZ types but not for nested ones, typically for now: array. This PR makes it work. ### Why are the changes needed? feature and data consistency ### Does this PR introduce _any_ user-facing change? preferTimestampNTZ jdbc option now works for array([timestamp|time|...]) in postgres ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #45418 from yaooqinn/SPARK-47316. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../sql/jdbc/PostgresIntegrationSuite.scala | 20 ++++---- .../sql/jdbc/v2/OracleIntegrationSuite.scala | 1 + .../apache/spark/sql/jdbc/v2/V2JDBCTest.scala | 5 +- .../datasources/jdbc/JdbcUtils.scala | 46 ++++++++++++------- .../spark/sql/jdbc/DatabricksDialect.scala | 5 +- .../apache/spark/sql/jdbc/JdbcDialects.scala | 2 +- .../spark/sql/jdbc/PostgresDialect.scala | 9 ++-- .../v2/jdbc/JDBCTableCatalogSuite.scala | 5 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 7 ++- 9 files changed, 63 insertions(+), 37 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 968ca09cb3d54..2d1c0314f27b4 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -416,17 +416,19 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { test("SPARK-43040: timestamp_ntz roundtrip test") { val prop = new Properties prop.setProperty("preferTimestampNTZ", "true") - - val sparkQuery = """ - |select - | timestamp_ntz'2020-12-10 11:22:33' as col0 - """.stripMargin - - val df_expected = sqlContext.sql(sparkQuery) + val df_expected = sql("select timestamp_ntz'2020-12-10 11:22:33' as col0") df_expected.write.jdbc(jdbcUrl, "timestamp_ntz_roundtrip", prop) + val df_actual = spark.read.jdbc(jdbcUrl, "timestamp_ntz_roundtrip", prop) + checkAnswer(df_actual, df_expected) + } - val df_actual = sqlContext.read.jdbc(jdbcUrl, "timestamp_ntz_roundtrip", prop) - assert(df_actual.collect()(0) == df_expected.collect()(0)) + test("SPARK-47316: timestamp_ntz_array roundtrip test") { + val prop = new Properties + prop.setProperty("preferTimestampNTZ", "true") + val df_expected = sql("select array(timestamp_ntz'2020-12-10 11:22:33') as col0") + df_expected.write.jdbc(jdbcUrl, "timestamp_ntz_array_roundtrip", prop) + val df_actual = spark.read.jdbc(jdbcUrl, "timestamp_ntz_array_roundtrip", prop) + checkAnswer(df_actual, df_expected) } test("SPARK-43267: user-defined column in array test") { diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala index 1badc110347d6..591147413486e 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/OracleIntegrationSuite.scala @@ -79,6 +79,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTes override val defaultMetadata: Metadata = new MetadataBuilder() .putLong("scale", 0) + .putBoolean("isTimestampNTZ", false) .putString(CHAR_VARCHAR_TYPE_STRING_METADATA_KEY, "varchar(255)") .build() diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala index 9a329def3bed2..8dd377f4a35fa 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/V2JDBCTest.scala @@ -49,7 +49,10 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu def notSupportsTableComment: Boolean = false - def defaultMetadata: Metadata = new MetadataBuilder().putLong("scale", 0).build() + def defaultMetadata: Metadata = new MetadataBuilder() + .putLong("scale", 0) + .putBoolean("isTimestampNTZ", false) + .build() def testUpdateColumnNullability(tbl: String): Unit = { sql(s"CREATE TABLE $catalogName.alt_table (ID STRING NOT NULL)") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 89ac615a3097e..b5e78ba32cd5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -36,8 +36,8 @@ import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils, DateTimeUtils, GenericArrayData} -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils, GenericArrayData} +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.connector.catalog.{Identifier, TableChange} import org.apache.spark.sql.connector.catalog.index.{SupportsIndex, TableIndex} import org.apache.spark.sql.connector.expressions.NamedReference @@ -302,7 +302,7 @@ object JdbcUtils extends Logging with SQLConfHelper { metadata.putBoolean("rowid", true) case _ => } - + metadata.putBoolean("isTimestampNTZ", isTimestampNTZ) val columnType = dialect.getCatalystType(dataType, typeName, fieldSize, metadata).getOrElse( getCatalystType(dataType, typeName, fieldSize, fieldScale, isSigned, isTimestampNTZ)) @@ -398,7 +398,7 @@ object JdbcUtils extends Logging with SQLConfHelper { // DateTimeUtils.fromJavaDate does not handle null value, so we need to check it. val dateVal = rs.getDate(pos + 1) if (dateVal != null) { - row.setInt(pos, DateTimeUtils.fromJavaDate(dateVal)) + row.setInt(pos, fromJavaDate(dateVal)) } else { row.update(pos, null) } @@ -477,8 +477,7 @@ object JdbcUtils extends Logging with SQLConfHelper { if (rawTime != null) { val localTimeMicro = TimeUnit.NANOSECONDS.toMicros( rawTime.toLocalTime().toNanoOfDay()) - val utcTimeMicro = DateTimeUtils.toUTCTime( - localTimeMicro, conf.sessionLocalTimeZone) + val utcTimeMicro = toUTCTime(localTimeMicro, conf.sessionLocalTimeZone) row.setLong(pos, utcTimeMicro) } else { row.update(pos, null) @@ -489,8 +488,7 @@ object JdbcUtils extends Logging with SQLConfHelper { (rs: ResultSet, row: InternalRow, pos: Int) => val t = rs.getTimestamp(pos + 1) if (t != null) { - row.setLong(pos, DateTimeUtils. - fromJavaTimestamp(dialect.convertJavaTimestampToTimestamp(t))) + row.setLong(pos, fromJavaTimestamp(dialect.convertJavaTimestampToTimestamp(t))) } else { row.update(pos, null) } @@ -499,8 +497,7 @@ object JdbcUtils extends Logging with SQLConfHelper { (rs: ResultSet, row: InternalRow, pos: Int) => val t = rs.getTimestamp(pos + 1) if (t != null) { - row.setLong(pos, - DateTimeUtils.localDateTimeToMicros(dialect.convertJavaTimestampToTimestampNTZ(t))) + row.setLong(pos, localDateTimeToMicros(dialect.convertJavaTimestampToTimestampNTZ(t))) } else { row.update(pos, null) } @@ -514,7 +511,14 @@ object JdbcUtils extends Logging with SQLConfHelper { case TimestampType => (array: Object) => array.asInstanceOf[Array[java.sql.Timestamp]].map { timestamp => - nullSafeConvert(timestamp, DateTimeUtils.fromJavaTimestamp) + nullSafeConvert(timestamp, fromJavaTimestamp) + } + + case TimestampNTZType => + (array: Object) => + array.asInstanceOf[Array[java.sql.Timestamp]].map { timestamp => + nullSafeConvert(timestamp, (t: java.sql.Timestamp) => + localDateTimeToMicros(dialect.convertJavaTimestampToTimestampNTZ(t))) } case StringType => @@ -526,7 +530,7 @@ object JdbcUtils extends Logging with SQLConfHelper { case DateType => (array: Object) => array.asInstanceOf[Array[java.sql.Date]].map { date => - nullSafeConvert(date, DateTimeUtils.fromJavaDate) + nullSafeConvert(date, fromJavaDate) } case dt: DecimalType => @@ -638,10 +642,20 @@ object JdbcUtils extends Logging with SQLConfHelper { // remove type length parameters from end of type name val typeName = getJdbcType(et, dialect).databaseTypeDefinition.split("\\(")(0) (stmt: PreparedStatement, row: Row, pos: Int) => - val array = conn.createArrayOf( - typeName, - row.getSeq[AnyRef](pos).toArray) - stmt.setArray(pos + 1, array) + et match { + case TimestampNTZType => + val array = row.getSeq[java.time.LocalDateTime](pos) + val arrayType = conn.createArrayOf( + typeName, + array.map(dialect.convertTimestampNTZToJavaTimestamp).toArray) + stmt.setArray(pos + 1, arrayType) + case _ => + val array = row.getSeq[AnyRef](pos) + val arrayType = conn.createArrayOf( + typeName, + array.toArray) + stmt.setArray(pos + 1, arrayType) + } case _ => (_: PreparedStatement, _: Row, pos: Int) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala index 697ddd2f0934a..c905374c16784 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DatabricksDialect.scala @@ -32,10 +32,7 @@ private case object DatabricksDialect extends JdbcDialect { } override def getCatalystType( - sqlType: Int, - typeName: String, - size: Int, - md: MetadataBuilder): Option[DataType] = { + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { sqlType match { case java.sql.Types.TINYINT => Some(ByteType) case java.sql.Types.SMALLINT => Some(ShortType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 06e59eea796b2..6621282647d4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -97,7 +97,7 @@ abstract class JdbcDialect extends Serializable with Logging { * or null if the default type mapping should be used. */ def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = None + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = None /** * Retrieve the jdbc / sql type for a given datatype. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 64e5c115ccecd..d19b5ba3e0eb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -65,15 +65,17 @@ private object PostgresDialect extends JdbcDialect with SQLConfHelper { Some(StringType) // sqlType is Types.VARCHAR } else if (sqlType == Types.ARRAY) { val scale = md.build().getLong("scale").toInt + val isTimestampNTZ = md.build().getBoolean("isTimestampNTZ") // postgres array type names start with underscore - toCatalystType(typeName.drop(1), size, scale).map(ArrayType(_)) + toCatalystType(typeName.drop(1), size, scale, isTimestampNTZ).map(ArrayType(_)) } else None } private def toCatalystType( typeName: String, precision: Int, - scale: Int): Option[DataType] = typeName match { + scale: Int, + isTimestampNTZ: Boolean): Option[DataType] = typeName match { case "bool" => Some(BooleanType) case "bit" => Some(BinaryType) case "int2" => Some(ShortType) @@ -89,7 +91,8 @@ private object PostgresDialect extends JdbcDialect with SQLConfHelper { "interval" | "pg_snapshot" => Some(StringType) case "bytea" => Some(BinaryType) - case "timestamp" | "timestamptz" | "time" | "timetz" => Some(TimestampType) + case "timestamp" | "timestamptz" | "time" | "timetz" => + Some(if (isTimestampNTZ) TimestampNTZType else TimestampType) case "date" => Some(DateType) case "numeric" | "decimal" if precision > 0 => Some(DecimalType.bounded(precision, scale)) case "numeric" | "decimal" => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 0088fab7d209e..6332c32c733a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -36,7 +36,10 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { val tempDir = Utils.createTempDir() val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" - val defaultMetadata = new MetadataBuilder().putLong("scale", 0).build() + val defaultMetadata = new MetadataBuilder() + .putLong("scale", 0) + .putBoolean("isTimestampNTZ", false) + .build() override def sparkConf: SparkConf = super.sparkConf .set("spark.sql.catalog.h2", classOf[JDBCTableCatalog].getName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 1faada03bc6c0..b8ca70e0b1752 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -78,7 +78,10 @@ class JDBCSuite extends QueryTest with SharedSparkSession { } } - val defaultMetadata = new MetadataBuilder().putLong("scale", 0).build() + val defaultMetadata = new MetadataBuilder() + .putLong("scale", 0) + .putBoolean("isTimestampNTZ", false) + .build() override def beforeAll(): Unit = { super.beforeAll() @@ -933,7 +936,7 @@ class JDBCSuite extends QueryTest with SharedSparkSession { test("PostgresDialect type mapping") { val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") - val md = new MetadataBuilder().putLong("scale", 0) + val md = new MetadataBuilder().putLong("scale", 0).putBoolean("isTimestampNTZ", false) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "json", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "jsonb", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.ARRAY, "_numeric", 0, md) == From 0a94e7c7b9a83ecfbca59f5b93532453f462500c Mon Sep 17 00:00:00 2001 From: Aleksandar Tomic Date: Fri, 8 Mar 2024 13:33:21 +0300 Subject: [PATCH 11/50] [SPARK-47302][SQL] Collate keyword as identifier ### What changes were proposed in this pull request? With this change we move away from using collation names as string literals and start treating them as identifiers, since that is the part of sql standard. Collation names are marked as multi part identifiers, since, in future, we will want to introduce user defined collations which can be part of nested namespaces in catalog. ### Why are the changes needed? Aligning with sql standard on collation syntax. ### Does this PR introduce _any_ user-facing change? Yes. Collations are still not a released feature. ### How was this patch tested? Existing tests are used. ### Was this patch authored or co-authored using generative AI tooling? Closes #45405 from dbatomic/collate_key_word_as_identifier. Lead-authored-by: Aleksandar Tomic Co-authored-by: Nikola Mandic Co-authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../queries/select_collated_string.json | 2 +- .../queries/select_collated_string.proto.bin | Bin 65 -> 63 bytes .../planner/SparkConnectProtoSuite.scala | 4 +- python/pyspark/sql/tests/test_types.py | 18 ++++---- python/pyspark/sql/types.py | 4 +- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../catalyst/parser/DataTypeAstBuilder.scala | 2 +- .../org/apache/spark/sql/types/DataType.scala | 2 +- .../apache/spark/sql/types/StringType.scala | 2 +- .../expressions/collationExpressions.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql-functions/sql-expression-schema.md | 2 +- .../analyzer-results/collations.sql.out | 26 +++++------ .../resources/sql-tests/inputs/collations.sql | 10 ++--- .../sql-tests/results/collations.sql.out | 22 ++++----- .../org/apache/spark/sql/CollationSuite.scala | 42 +++++++++++------- .../errors/QueryCompilationErrorsSuite.scala | 6 +-- 17 files changed, 79 insertions(+), 69 deletions(-) diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json index db065b36e3458..00644e072190e 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json @@ -8,7 +8,7 @@ "planId": "0" }, "localRelation": { - "schema": "struct\u003cs:string COLLATE \u0027UCS_BASIC_LCASE\u0027\u003e" + "schema": "struct\u003cs:string COLLATE UCS_BASIC_LCASE\u003e" } }, "expressions": [{ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin index 3a5661e54ce0aa932112cb2ef1ff7a12687b303d..b247463daf43d9162d0107fa10f9d5ab24d08f57 100644 GIT binary patch delta 46 zcmZ>?7vN$NV3e}tG62#HQCdP869pv1L!E=;og9Nbo#TC+9fMu%q*#TRxfqK9t=$Mn delta 48 zcmcC_6yRbKV3e}vG6K>JQ943e69ptB)kB?w Seq( - s"select 'abc' collate '$collationName'", - s"select collation('abc' collate '$collationName')").map(query => + s"select 'abc' collate $collationName", + s"select collation('abc' collate $collationName)").map(query => comparePlans(connect.sql(query), spark.sql(query)))) } diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index e0f81dc08743f..a0dfdce1a96e8 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -862,15 +862,15 @@ def test_parse_datatype_string(self): if k != "varchar" and k != "char": self.assertEqual(t(), _parse_datatype_string(k)) self.assertEqual(IntegerType(), _parse_datatype_string("int")) - self.assertEqual(StringType(), _parse_datatype_string("string COLLATE 'UCS_BASIC'")) + self.assertEqual(StringType(), _parse_datatype_string("string COLLATE UCS_BASIC")) self.assertEqual(StringType(0), _parse_datatype_string("string")) - self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE 'UCS_BASIC'")) - self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE 'UCS_BASIC'")) - self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE'UCS_BASIC'")) - self.assertEqual(StringType(1), _parse_datatype_string("string COLLATE 'UCS_BASIC_LCASE'")) - self.assertEqual(StringType(1), _parse_datatype_string("string COLLATE 'UCS_BASIC_LCASE'")) - self.assertEqual(StringType(2), _parse_datatype_string("string COLLATE 'UNICODE'")) - self.assertEqual(StringType(3), _parse_datatype_string("string COLLATE 'UNICODE_CI'")) + self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UCS_BASIC")) + self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UCS_BASIC")) + self.assertEqual(StringType(1), _parse_datatype_string("string COLLATE UCS_BASIC_LCASE")) + self.assertEqual(StringType(2), _parse_datatype_string("string COLLATE UNICODE")) + self.assertEqual(StringType(2), _parse_datatype_string("string COLLATE `UNICODE`")) + self.assertEqual(StringType(3), _parse_datatype_string("string COLLATE UNICODE_CI")) + self.assertEqual(StringType(3), _parse_datatype_string("string COLLATE `UNICODE_CI`")) self.assertEqual(CharType(1), _parse_datatype_string("char(1)")) self.assertEqual(CharType(10), _parse_datatype_string("char( 10 )")) self.assertEqual(CharType(11), _parse_datatype_string("char( 11)")) @@ -1410,7 +1410,7 @@ def test_from_ddl(self): def test_collated_string(self): dfs = [ - self.spark.sql("SELECT 'abc' collate 'UCS_BASIC_LCASE'"), + self.spark.sql("SELECT 'abc' collate UCS_BASIC_LCASE"), self.spark.createDataFrame([], StructType([StructField("id", StringType(1))])), ] for df in dfs: diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index c4647b90767f5..a30f41ae40239 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -261,7 +261,7 @@ def __init__(self, collationId: int = 0): def collationIdToName(self) -> str: return ( - " COLLATE '%s'" % StringType.collationNames[self.collationId] + " COLLATE %s" % StringType.collationNames[self.collationId] if self.collationId != 0 else "" ) @@ -1486,7 +1486,7 @@ def __eq__(self, other: Any) -> bool: (v.typeName(), v) for v in _complex_types ) -_COLLATED_STRING = re.compile(r"string\s+COLLATE\s+'([\w_]+)'") +_COLLATED_STRING = re.compile(r"string\s+COLLATE\s+([\w_]+|`[\w_]`)") _LENGTH_CHAR = re.compile(r"char\(\s*(\d+)\s*\)") _LENGTH_VARCHAR = re.compile(r"varchar\(\s*(\d+)\s*\)") _FIXED_DECIMAL = re.compile(r"decimal\(\s*(\d+)\s*,\s*(-?\d+)\s*\)") diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index ca01de4ffdc2e..801cc62491a2d 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1096,7 +1096,7 @@ colPosition ; collateClause - : COLLATE collationName=stringLit + : COLLATE collationName=identifier ; type diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala index 0d2822e13efcb..5c2df6a6e9d9a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/DataTypeAstBuilder.scala @@ -218,6 +218,6 @@ class DataTypeAstBuilder extends SqlBaseParserBaseVisitor[AnyRef] { * Returns a collation name. */ override def visitCollateClause(ctx: CollateClauseContext): String = withOrigin(ctx) { - string(visitStringLit(ctx.stringLit)) + ctx.identifier.getText } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala index efaf6e6bfd6a6..2ffe0ba379dd4 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -117,7 +117,7 @@ object DataType { private val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r private val CHAR_TYPE = """char\(\s*(\d+)\s*\)""".r private val VARCHAR_TYPE = """varchar\(\s*(\d+)\s*\)""".r - private val COLLATED_STRING_TYPE = """string\s+COLLATE\s+'([\w_]+)'""".r + private val COLLATED_STRING_TYPE = """string\s+COLLATE\s+([\w_]+|`[\w_]`)""".r def fromDDL(ddl: String): DataType = { parseTypeWithFallback( diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index 313f525742ae9..b434c98edd2a5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -47,7 +47,7 @@ class StringType private(val collationId: Int) extends AtomicType with Serializa */ override def typeName: String = if (isDefaultCollation) "string" - else s"string COLLATE '${CollationFactory.fetchCollation(collationId).collationName}'" + else s"string COLLATE ${CollationFactory.fetchCollation(collationId).collationName}" override def equals(obj: Any): Boolean = obj.isInstanceOf[StringType] && obj.asInstanceOf[StringType].collationId == collationId diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala index e51d9a67b1663..99237d978deea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.types._ Examples: > SET spark.sql.collation.enabled=true; spark.sql.collation.enabled true - > SELECT COLLATION('Spark SQL' _FUNC_ 'UCS_BASIC_LCASE'); + > SELECT COLLATION('Spark SQL' _FUNC_ UCS_BASIC_LCASE); UCS_BASIC_LCASE > SET spark.sql.collation.enabled=false; spark.sql.collation.enabled false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 48d8b8ad8f4d1..9fed85b2a2a7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2194,7 +2194,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { if (!SQLConf.get.collationEnabled) { throw QueryCompilationErrors.collationNotEnabledError() } - string(visitStringLit(ctx.stringLit)) + ctx.identifier.getText } /** diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index e20db3b49589c..33c26fe3c036b 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -81,7 +81,7 @@ | org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | struct | | org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | struct | | org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT coalesce(NULL, 1, NULL) | struct | -| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate 'UCS_BASIC_LCASE') | struct | +| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UCS_BASIC_LCASE) | struct | | org.apache.spark.sql.catalyst.expressions.Collation | collation | SELECT collation('Spark SQL') | struct | | org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT concat('Spark', 'SQL') | struct | | org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT concat_ws(' ', 'Spark', 'SQL') | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index 76f20e7f8dcbd..794185c5cba44 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -1,6 +1,6 @@ -- Automatically generated by SQLQueryTestSuite -- !query -create table t1(ucs_basic string collate 'ucs_basic', ucs_basic_lcase string collate 'ucs_basic_lcase') using parquet +create table t1(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false @@ -9,7 +9,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false insert into t1 values('aaa', 'aaa') -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE 'UCS_BASIC_LCASE') AS ucs_basic_lcase#x] ++- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -17,7 +17,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d insert into t1 values('AAA', 'AAA') -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE 'UCS_BASIC_LCASE') AS ucs_basic_lcase#x] ++- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -25,7 +25,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d insert into t1 values('bbb', 'bbb') -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE 'UCS_BASIC_LCASE') AS ucs_basic_lcase#x] ++- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -33,7 +33,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d insert into t1 values('BBB', 'BBB') -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE 'UCS_BASIC_LCASE') AS ucs_basic_lcase#x] ++- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -63,7 +63,7 @@ Project [ucs_basic#x, ucs_basic_lcase#x] -- !query -select * from t1 where ucs_basic_lcase = 'aaa' collate 'ucs_basic_lcase' +select * from t1 where ucs_basic_lcase = 'aaa' collate ucs_basic_lcase -- !query analysis Project [ucs_basic#x, ucs_basic_lcase#x] +- Filter (ucs_basic_lcase#x = collate(aaa, ucs_basic_lcase)) @@ -81,7 +81,7 @@ Project [ucs_basic#x, ucs_basic_lcase#x] -- !query -select * from t1 where ucs_basic_lcase < 'bbb' collate 'ucs_basic_lcase' +select * from t1 where ucs_basic_lcase < 'bbb' collate ucs_basic_lcase -- !query analysis Project [ucs_basic#x, ucs_basic_lcase#x] +- Filter (ucs_basic_lcase#x < collate(bbb, ucs_basic_lcase)) @@ -103,7 +103,7 @@ Project [ucs_basic#x, ucs_basic_lcase#x] -- !query -create table t2(ucs_basic string collate 'ucs_basic', ucs_basic_lcase string collate 'ucs_basic_lcase') using parquet +create table t2(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false @@ -112,7 +112,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false insert into t2 values('aaa', 'aaa') -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE 'UCS_BASIC_LCASE') AS ucs_basic_lcase#x] ++- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -120,7 +120,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d insert into t2 values('bbb', 'bbb') -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE 'UCS_BASIC_LCASE') AS ucs_basic_lcase#x] ++- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -150,7 +150,7 @@ DropTable false, false -- !query -create table t1 (c1 struct) USING PARQUET +create table t1 (c1 struct) USING PARQUET -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false @@ -159,7 +159,7 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false INSERT INTO t1 VALUES (named_struct('ucs_basic', 'aaa', 'ucs_basic_lcase', 'aaa')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] -+- Project [named_struct(ucs_basic, col1#x.ucs_basic, ucs_basic_lcase, cast(col1#x.ucs_basic_lcase as string COLLATE 'UCS_BASIC_LCASE')) AS c1#x] ++- Project [named_struct(ucs_basic, col1#x.ucs_basic, ucs_basic_lcase, cast(col1#x.ucs_basic_lcase as string COLLATE UCS_BASIC_LCASE)) AS c1#x] +- LocalRelation [col1#x] @@ -167,7 +167,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d INSERT INTO t1 VALUES (named_struct('ucs_basic', 'AAA', 'ucs_basic_lcase', 'AAA')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] -+- Project [named_struct(ucs_basic, col1#x.ucs_basic, ucs_basic_lcase, cast(col1#x.ucs_basic_lcase as string COLLATE 'UCS_BASIC_LCASE')) AS c1#x] ++- Project [named_struct(ucs_basic, col1#x.ucs_basic, ucs_basic_lcase, cast(col1#x.ucs_basic_lcase as string COLLATE UCS_BASIC_LCASE)) AS c1#x] +- LocalRelation [col1#x] diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index 96e82c695069a..91a159b39aeea 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -1,7 +1,7 @@ -- test cases for collation support -- Create a test table with data -create table t1(ucs_basic string collate 'ucs_basic', ucs_basic_lcase string collate 'ucs_basic_lcase') using parquet; +create table t1(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet; insert into t1 values('aaa', 'aaa'); insert into t1 values('AAA', 'AAA'); insert into t1 values('bbb', 'bbb'); @@ -17,19 +17,19 @@ select count(*) from t1 group by ucs_basic_lcase; select * from t1 where ucs_basic = 'aaa'; -- filter equal ucs_basic_lcase -select * from t1 where ucs_basic_lcase = 'aaa' collate 'ucs_basic_lcase'; +select * from t1 where ucs_basic_lcase = 'aaa' collate ucs_basic_lcase; -- filter less then ucs_basic select * from t1 where ucs_basic < 'bbb'; -- filter less then ucs_basic_lcase -select * from t1 where ucs_basic_lcase < 'bbb' collate 'ucs_basic_lcase'; +select * from t1 where ucs_basic_lcase < 'bbb' collate ucs_basic_lcase; -- inner join select l.ucs_basic, r.ucs_basic_lcase from t1 l join t1 r on l.ucs_basic_lcase = r.ucs_basic_lcase; -- create second table for anti-join -create table t2(ucs_basic string collate 'ucs_basic', ucs_basic_lcase string collate 'ucs_basic_lcase') using parquet; +create table t2(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet; insert into t2 values('aaa', 'aaa'); insert into t2 values('bbb', 'bbb'); @@ -40,7 +40,7 @@ drop table t2; drop table t1; -- create table with struct field -create table t1 (c1 struct) USING PARQUET; +create table t1 (c1 struct) USING PARQUET; INSERT INTO t1 VALUES (named_struct('ucs_basic', 'aaa', 'ucs_basic_lcase', 'aaa')); INSERT INTO t1 VALUES (named_struct('ucs_basic', 'AAA', 'ucs_basic_lcase', 'AAA')); diff --git a/sql/core/src/test/resources/sql-tests/results/collations.sql.out b/sql/core/src/test/resources/sql-tests/results/collations.sql.out index 49d6b94cae963..7814ae59173dd 100644 --- a/sql/core/src/test/resources/sql-tests/results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/collations.sql.out @@ -1,6 +1,6 @@ -- Automatically generated by SQLQueryTestSuite -- !query -create table t1(ucs_basic string collate 'ucs_basic', ucs_basic_lcase string collate 'ucs_basic_lcase') using parquet +create table t1(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet -- !query schema struct<> -- !query output @@ -62,15 +62,15 @@ struct -- !query select * from t1 where ucs_basic = 'aaa' -- !query schema -struct +struct -- !query output aaa aaa -- !query -select * from t1 where ucs_basic_lcase = 'aaa' collate 'ucs_basic_lcase' +select * from t1 where ucs_basic_lcase = 'aaa' collate ucs_basic_lcase -- !query schema -struct +struct -- !query output AAA AAA aaa aaa @@ -79,7 +79,7 @@ aaa aaa -- !query select * from t1 where ucs_basic < 'bbb' -- !query schema -struct +struct -- !query output AAA AAA BBB BBB @@ -87,9 +87,9 @@ aaa aaa -- !query -select * from t1 where ucs_basic_lcase < 'bbb' collate 'ucs_basic_lcase' +select * from t1 where ucs_basic_lcase < 'bbb' collate ucs_basic_lcase -- !query schema -struct +struct -- !query output AAA AAA aaa aaa @@ -98,7 +98,7 @@ aaa aaa -- !query select l.ucs_basic, r.ucs_basic_lcase from t1 l join t1 r on l.ucs_basic_lcase = r.ucs_basic_lcase -- !query schema -struct +struct -- !query output AAA AAA AAA aaa @@ -111,7 +111,7 @@ bbb bbb -- !query -create table t2(ucs_basic string collate 'ucs_basic', ucs_basic_lcase string collate 'ucs_basic_lcase') using parquet +create table t2(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet -- !query schema struct<> -- !query output @@ -137,7 +137,7 @@ struct<> -- !query select * from t1 anti join t2 on t1.ucs_basic_lcase = t2.ucs_basic_lcase -- !query schema -struct +struct -- !query output @@ -159,7 +159,7 @@ struct<> -- !query -create table t1 (c1 struct) USING PARQUET +create table t1 (c1 struct) USING PARQUET -- !query schema struct<> -- !query output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index 3632c06bfe7c0..6f183e60f5895 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -37,18 +37,18 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("collate returns proper type") { Seq("ucs_basic", "ucs_basic_lcase", "unicode", "unicode_ci").foreach { collationName => - checkAnswer(sql(s"select 'aaa' collate '$collationName'"), Row("aaa")) + checkAnswer(sql(s"select 'aaa' collate $collationName"), Row("aaa")) val collationId = CollationFactory.collationNameToId(collationName) - assert(sql(s"select 'aaa' collate '$collationName'").schema(0).dataType + assert(sql(s"select 'aaa' collate $collationName").schema(0).dataType == StringType(collationId)) } } test("collation name is case insensitive") { Seq("uCs_BasIc", "uCs_baSic_Lcase", "uNicOde", "UNICODE_ci").foreach { collationName => - checkAnswer(sql(s"select 'aaa' collate '$collationName'"), Row("aaa")) + checkAnswer(sql(s"select 'aaa' collate $collationName"), Row("aaa")) val collationId = CollationFactory.collationNameToId(collationName) - assert(sql(s"select 'aaa' collate '$collationName'").schema(0).dataType + assert(sql(s"select 'aaa' collate $collationName").schema(0).dataType == StringType(collationId)) } } @@ -56,7 +56,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("collation expression returns name of collation") { Seq("ucs_basic", "ucs_basic_lcase", "unicode", "unicode_ci").foreach { collationName => checkAnswer( - sql(s"select collation('aaa' collate '$collationName')"), Row(collationName.toUpperCase())) + sql(s"select collation('aaa' collate $collationName)"), Row(collationName.toUpperCase())) } } @@ -132,7 +132,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("invalid collation name throws exception") { checkError( - exception = intercept[SparkException] { sql("select 'aaa' collate 'UCS_BASIS'") }, + exception = intercept[SparkException] { sql("select 'aaa' collate UCS_BASIS") }, errorClass = "COLLATION_INVALID_NAME", sqlState = "42704", parameters = Map("proposal" -> "UCS_BASIC", "collationName" -> "UCS_BASIS")) @@ -153,7 +153,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ).foreach { case (collationName, left, right, expected) => checkAnswer( - sql(s"select '$left' collate '$collationName' = '$right' collate '$collationName'"), + sql(s"select '$left' collate $collationName = '$right' collate $collationName"), Row(expected)) checkAnswer( sql(s"select collate('$left', '$collationName') = collate('$right', '$collationName')"), @@ -178,7 +178,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ).foreach { case (collationName, left, right, expected) => checkAnswer( - sql(s"select '$left' collate '$collationName' < '$right' collate '$collationName'"), + sql(s"select '$left' collate $collationName < '$right' collate $collationName"), Row(expected)) checkAnswer( sql(s"select collate('$left', '$collationName') < collate('$right', '$collationName')"), @@ -420,9 +420,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val tableNameBinary = "T_BINARY" withTable(tableNameNonBinary) { withTable(tableNameBinary) { - sql(s"CREATE TABLE $tableNameNonBinary (c STRING COLLATE 'UCS_BASIC_LCASE') USING PARQUET") + sql(s"CREATE TABLE $tableNameNonBinary (c STRING COLLATE UCS_BASIC_LCASE) USING PARQUET") sql(s"INSERT INTO $tableNameNonBinary VALUES ('aaa')") - sql(s"CREATE TABLE $tableNameBinary (c STRING COLLATE 'UCS_BASIC') USING PARQUET") + sql(s"CREATE TABLE $tableNameBinary (c STRING COLLATE UCS_BASIC) USING PARQUET") sql(s"INSERT INTO $tableNameBinary VALUES ('aaa')") val dfNonBinary = sql(s"SELECT COUNT(*), c FROM $tableNameNonBinary GROUP BY c") @@ -438,6 +438,16 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } + test("text writing to parquet with collation enclosed with backticks") { + withTempPath{ path => + sql(s"select 'a' COLLATE `UNICODE`").write.parquet(path.getAbsolutePath) + + checkAnswer( + spark.read.parquet(path.getAbsolutePath), + Row("a")) + } + } + test("create table with collation") { val tableName = "parquet_dummy_tbl" val collationName = "UCS_BASIC_LCASE" @@ -446,7 +456,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { withTable(tableName) { sql( s""" - |CREATE TABLE $tableName (c1 STRING COLLATE '$collationName') + |CREATE TABLE $tableName (c1 STRING COLLATE $collationName) |USING PARQUET |""".stripMargin) @@ -467,7 +477,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql( s""" |CREATE TABLE $tableName - |(c1 STRUCT) + |(c1 STRUCT) |USING PARQUET |""".stripMargin) @@ -502,7 +512,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql( s""" |ALTER TABLE $tableName - |ADD COLUMN c2 STRING COLLATE '$collationName' + |ADD COLUMN c2 STRING COLLATE $collationName |""".stripMargin) sql(s"INSERT INTO $tableName VALUES ('aaa', 'aaa')") @@ -522,7 +532,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { withTable(tableName) { sql( s""" - |CREATE TABLE $tableName (c1 string COLLATE '$collationName') + |CREATE TABLE $tableName (c1 string COLLATE $collationName) |USING $v2Source |""".stripMargin) @@ -552,7 +562,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql( s""" |CREATE TABLE $tableName - |(id INT, c1 STRING COLLATE 'UNICODE', c2 string) + |(id INT, c1 STRING COLLATE UNICODE, c2 string) |USING parquet |PARTITIONED BY (${partitionColumns.mkString(",")}) |""".stripMargin) @@ -570,7 +580,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { createTable(partitionColumns: _*) }, errorClass = "INVALID_PARTITION_COLUMN_DATA_TYPE", - parameters = Map("type" -> "\"STRING COLLATE 'UNICODE'\"") + parameters = Map("type" -> "\"STRING COLLATE UNICODE\"") ); } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index 72250f9a7fdc2..c9198c86c7202 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -955,9 +955,9 @@ class QueryCompilationErrorsSuite test("SPARK-47102: the collation feature is off without collate builder call") { withSQLConf(SQLConf.COLLATION_ENABLED.key -> "false") { Seq( - "CREATE TABLE t(col STRING COLLATE 'UNICODE_CI') USING parquet", - "CREATE TABLE t(col STRING COLLATE 'UNKNOWN_COLLATION_STRING') USING parquet", - "SELECT 'aaa' COLLATE 'UNICODE_CI'", + "CREATE TABLE t(col STRING COLLATE UNICODE_CI) USING parquet", + "CREATE TABLE t(col STRING COLLATE UNKNOWN_COLLATION_STRING) USING parquet", + "SELECT 'aaa' COLLATE UNICODE_CI", "select collation('aaa')" ).foreach { sqlText => checkError( From 640ed4fad32f1042d564166ddca2609190fb6a96 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 8 Mar 2024 20:52:22 +0900 Subject: [PATCH 12/50] [SPARK-47322][PYTHON][CONNECT] Make `withColumnsRenamed` column names duplication handling consistent with `withColumnRenamed` ### What changes were proposed in this pull request? Make `withColumnsRenamed` duplicated column name handling consistent with `withColumnRenamed` ### Why are the changes needed? `withColumnsRenamed` checks the column names duplication of output dataframe, this is not consistent with `withColumnRenamed`: 1, `withColumnRenamed` doesn't do this check, and support output a dataframe with duplicated column names; 2, when the input dataframe has duplicated column names, `withColumnsRenamed` always fail, even if the columns with the same name are not touched at all: ``` In [8]: df1 = spark.createDataFrame([(1, "id2"),], ["id", "value"]) ...: df2 = spark.createDataFrame([(1, 'x', 'id1'), ], ["id", 'a', "value"]) ...: join = df2.join(df1, on=['id'], how='left') ...: join Out[8]: DataFrame[id: bigint, a: string, value: string, value: string] In [9]: join.withColumnRenamed('id', 'value') Out[9]: DataFrame[value: bigint, a: string, value: string, value: string] In [10]: join.withColumnsRenamed({'id' : 'value'}) ... AnalysisException: [COLUMN_ALREADY_EXISTS] The column `value` already exists. Choose another name or rename the existing column. SQLSTATE: 42711 In [11]: join.withColumnRenamed('a', 'b') Out[11]: DataFrame[id: bigint, b: string, value: string, value: string] In [12]: join.withColumnsRenamed({'a' : 'b'}) ... AnalysisException: [COLUMN_ALREADY_EXISTS] The column `value` already exists. Choose another name or rename the existing column. SQLSTATE: 42711 In [13]: join.withColumnRenamed('x', 'y') Out[13]: DataFrame[id: bigint, a: string, value: string, value: string] In [14]: join.withColumnsRenamed({'x' : 'y'}) AnalysisException: [COLUMN_ALREADY_EXISTS] The column `value` already exists. Choose another name or rename the existing column. SQLSTATE: 42711 In [15]: join.withColumnRenamed('value', 'new_value') Out[15]: DataFrame[id: bigint, a: string, new_value: string, new_value: string] In [16]: join.withColumnsRenamed({'value' : 'new_value'}) AnalysisException: [COLUMN_ALREADY_EXISTS] The column `new_value` already exists. Choose another name or rename the existing column. SQLSTATE: 42711 ``` ### Does this PR introduce _any_ user-facing change? yes ### How was this patch tested? updated tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #45431 from zhengruifeng/connect_renames. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .../planner/SparkConnectProtoSuite.scala | 9 ----- python/pyspark/sql/tests/test_dataframe.py | 22 +++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 3 -- .../org/apache/spark/sql/DataFrameSuite.scala | 39 ------------------- 4 files changed, 22 insertions(+), 51 deletions(-) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index 1b50936d935ad..b989f5027cf92 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -566,15 +566,6 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { comparePlans( connectTestRelation.withColumnsRenamed(Map("id" -> "id1", "id" -> "id2")), sparkTestRelation.withColumnsRenamed(Map("id" -> "id1", "id" -> "id2"))) - - checkError( - exception = intercept[AnalysisException] { - transform( - connectTestRelation.withColumnsRenamed( - Map("id" -> "duplicatedCol", "name" -> "duplicatedCol"))) - }, - errorClass = "COLUMN_ALREADY_EXISTS", - parameters = Map("columnName" -> "`duplicatedcol`")) } test("Writes fails without path or table") { diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index ad7a3b8dfc96a..38310f12a4a72 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -231,6 +231,28 @@ def test_with_columns_renamed(self): message_parameters={"arg_name": "colsMap", "arg_type": "tuple"}, ) + def test_with_columns_renamed_with_duplicated_names(self): + df1 = self.spark.createDataFrame([(1, "v1")], ["id", "value"]) + df2 = self.spark.createDataFrame([(1, "x", "v2")], ["id", "a", "value"]) + join = df2.join(df1, on=["id"], how="left") + + self.assertEqual( + join.withColumnRenamed("id", "value").columns, + join.withColumnsRenamed({"id": "value"}).columns, + ) + self.assertEqual( + join.withColumnRenamed("a", "b").columns, + join.withColumnsRenamed({"a": "b"}).columns, + ) + self.assertEqual( + join.withColumnRenamed("value", "new_value").columns, + join.withColumnsRenamed({"value": "new_value"}).columns, + ) + self.assertEqual( + join.withColumnRenamed("x", "y").columns, + join.withColumnsRenamed({"x": "y"}).columns, + ) + def test_ordering_of_with_columns_renamed(self): df = self.spark.range(10) 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 189be1d6a30d2..f3bf6119659d1 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 @@ -2932,9 +2932,6 @@ class Dataset[T] private[sql]( } ) } - SchemaUtils.checkColumnNameDuplication( - projectList.map(_.name), - sparkSession.sessionState.conf.caseSensitiveAnalysis) withPlan(Project(projectList, logicalPlan)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 1dc367fa6bf68..6b34a6412cc0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -789,45 +789,6 @@ class DataFrameSuite extends QueryTest assert(df.columns === Array("key", "value", "renamed1", "renamed2")) } - test("SPARK-40311: withColumnsRenamed case sensitive") { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val df = testData.toDF().withColumns(Seq("newCol1", "newCOL2"), - Seq(col("key") + 1, col("key") + 2)) - .withColumnsRenamed(Map("newCol1" -> "renamed1", "newCol2" -> "renamed2")) - checkAnswer( - df, - testData.collect().map { case Row(key: Int, value: String) => - Row(key, value, key + 1, key + 2) - }.toSeq) - assert(df.columns === Array("key", "value", "renamed1", "newCOL2")) - } - } - - test("SPARK-40311: withColumnsRenamed duplicate column names simple") { - checkError( - exception = intercept[AnalysisException] { - person.withColumnsRenamed(Map("id" -> "renamed", "name" -> "renamed")) - }, - errorClass = "COLUMN_ALREADY_EXISTS", - parameters = Map("columnName" -> "`renamed`")) - } - - test("SPARK-40311: withColumnsRenamed duplicate column names simple case sensitive") { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val df = person.withColumnsRenamed(Map("id" -> "renamed", "name" -> "Renamed")) - assert(df.columns === Array("renamed", "Renamed", "age")) - } - } - - test("SPARK-40311: withColumnsRenamed duplicate column names indirect") { - checkError( - exception = intercept[AnalysisException] { - person.withColumnsRenamed(Map("id" -> "renamed1", "renamed1" -> "age")) - }, - errorClass = "COLUMN_ALREADY_EXISTS", - parameters = Map("columnName" -> "`age`")) - } - test("SPARK-46260: withColumnsRenamed should respect the Map ordering") { val df = spark.range(10).toDF() assert(df.withColumnsRenamed(ListMap("id" -> "a", "a" -> "b")).columns === Array("b")) From 76b1c122cb7d77e8f175b25b935b9296a669d5d8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 8 Mar 2024 13:31:10 -0800 Subject: [PATCH 13/50] [SPARK-44115][BUILD] Upgrade Apache ORC to 2.0.0 ### What changes were proposed in this pull request? This PR aims to Upgrade Apache ORC to 2.0.0 for Apache Spark 4.0.0. Apache ORC community has 3-year support policy which is longer than Apache Spark. It's aligned like the following. - Apache ORC 2.0.x <-> Apache Spark 4.0.x - Apache ORC 1.9.x <-> Apache Spark 3.5.x - Apache ORC 1.8.x <-> Apache Spark 3.4.x - Apache ORC 1.7.x (Supported) <-> Apache Spark 3.3.x (End-Of-Support) ### Why are the changes needed? **Release Note** - https://github.com/apache/orc/releases/tag/v2.0.0 **Milestone** - https://github.com/apache/orc/milestone/20?closed=1 - https://github.com/apache/orc/pull/1728 - https://github.com/apache/orc/issues/1801 - https://github.com/apache/orc/issues/1498 - https://github.com/apache/orc/pull/1627 - https://github.com/apache/orc/issues/1497 - https://github.com/apache/orc/pull/1509 - https://github.com/apache/orc/pull/1554 - https://github.com/apache/orc/pull/1708 - https://github.com/apache/orc/pull/1733 - https://github.com/apache/orc/pull/1760 - https://github.com/apache/orc/pull/1743 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45443 from dongjoon-hyun/SPARK-44115. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 7 ++++--- pom.xml | 17 ++++++++++++++++- sql/core/pom.xml | 5 +++++ 3 files changed, 25 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 7e56e89144359..6b357b3e4b70f 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -227,9 +227,10 @@ opencsv/2.3//opencsv-2.3.jar opentracing-api/0.33.0//opentracing-api-0.33.0.jar opentracing-noop/0.33.0//opentracing-noop-0.33.0.jar opentracing-util/0.33.0//opentracing-util-0.33.0.jar -orc-core/1.9.2/shaded-protobuf/orc-core-1.9.2-shaded-protobuf.jar -orc-mapreduce/1.9.2/shaded-protobuf/orc-mapreduce-1.9.2-shaded-protobuf.jar -orc-shims/1.9.2//orc-shims-1.9.2.jar +orc-core/2.0.0/shaded-protobuf/orc-core-2.0.0-shaded-protobuf.jar +orc-format/1.0.0/shaded-protobuf/orc-format-1.0.0-shaded-protobuf.jar +orc-mapreduce/2.0.0/shaded-protobuf/orc-mapreduce-2.0.0-shaded-protobuf.jar +orc-shims/2.0.0//orc-shims-2.0.0.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index 9f1c9ed13f23e..404f37be1b5ae 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 10.16.1.1 1.13.1 - 1.9.2 + 2.0.0 shaded-protobuf 11.0.20 5.0.0 @@ -2593,6 +2593,13 @@ + + org.apache.orc + orc-format + 1.0.0 + ${orc.classifier} + ${orc.deps.scope} + org.apache.orc orc-core @@ -2600,6 +2607,14 @@ ${orc.classifier} ${orc.deps.scope} + + org.apache.orc + orc-format + + + com.aayushatharva.brotli4j + brotli4j + org.apache.hadoop hadoop-common diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0ad9e0f690c75..05f906206e5e2 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -93,6 +93,11 @@ org.scala-lang.modules scala-parallel-collections_${scala.binary.version} + + org.apache.orc + orc-format + ${orc.classifier} + org.apache.orc orc-core From 264e00ee12bbbd822e52fa8ce79692c60f531495 Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Mon, 11 Mar 2024 00:25:55 +0500 Subject: [PATCH 14/50] [MINOR][SQL][TEST] Moving tests to related suites ### What changes were proposed in this pull request? Tests from `QueryCompilationErrorsSuite` were moved to `DDLSuite` and `JDBCTableCatalogSuite`. ### Why are the changes needed? We should move tests to related test suites in order to improve testing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Corresponding Suites succeed. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45439 from mihailom-db/SPARK-47326. Authored-by: Mihailo Milosevic Signed-off-by: Max Gekk --- .../errors/QueryCompilationErrorsSuite.scala | 74 ------------------- .../sql/execution/command/DDLSuite.scala | 17 +++++ .../v2/jdbc/JDBCTableCatalogSuite.scala | 56 ++++++++++++++ 3 files changed, 73 insertions(+), 74 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index c9198c86c7202..4574d3328d48a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -25,13 +25,11 @@ import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test} import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter -import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils case class StringLongClass(a: String, b: Long) @@ -817,78 +815,6 @@ class QueryCompilationErrorsSuite parameters = Map("extraction" -> "\"array(test)\"")) } - test("CREATE NAMESPACE with LOCATION for JDBC catalog should throw an error") { - withTempDir { tempDir => - val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" - Utils.classForName("org.h2.Driver") - withSQLConf( - "spark.sql.catalog.h2" -> classOf[JDBCTableCatalog].getName, - "spark.sql.catalog.h2.url" -> url, - "spark.sql.catalog.h2.driver" -> "org.h2.Driver") { - checkError( - exception = intercept[AnalysisException] { - sql("CREATE NAMESPACE h2.test_namespace LOCATION './samplepath'") - }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND", - sqlState = "0A000", - parameters = Map("cmd" -> toSQLStmt("CREATE NAMESPACE ... LOCATION ..."))) - } - } - } - - test("ALTER NAMESPACE with property other than COMMENT " + - "for JDBC catalog should throw an exception") { - withTempDir { tempDir => - val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" - Utils.classForName("org.h2.Driver") - withSQLConf( - "spark.sql.catalog.h2" -> classOf[JDBCTableCatalog].getName, - "spark.sql.catalog.h2.url" -> url, - "spark.sql.catalog.h2.driver" -> "org.h2.Driver") { - val namespace = "h2.test_namespace" - withNamespace(namespace) { - sql(s"CREATE NAMESPACE $namespace") - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER NAMESPACE h2.test_namespace SET LOCATION '/tmp/loc_test_2'") - }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", - sqlState = "0A000", - parameters = Map( - "cmd" -> toSQLStmt("SET NAMESPACE"), - "property" -> toSQLConf("location"))) - - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER NAMESPACE h2.test_namespace SET PROPERTIES('a'='b')") - }, - errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", - sqlState = "0A000", - parameters = Map( - "cmd" -> toSQLStmt("SET NAMESPACE"), - "property" -> toSQLConf("a"))) - } - } - } - } - - test("ALTER TABLE UNSET nonexistent property should throw an exception") { - val tableName = "test_table" - withTable(tableName) { - sql(s"CREATE TABLE $tableName (a STRING, b INT) USING parquet") - - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER TABLE $tableName UNSET TBLPROPERTIES ('test_prop1', 'test_prop2', 'comment')") - }, - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map( - "properties" -> "`test_prop1`, `test_prop2`", - "table" -> "`spark_catalog`.`default`.`test_table`") - ) - } - } - test("SPARK-43841: Unresolved attribute in select of full outer join with USING") { withTempView("v1", "v2") { sql("create or replace temp view v1 as values (1, 2) as (c1, c2)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e47858a690770..e8af606d797e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -335,6 +335,23 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { testUnsetProperties(isDatasourceTable = true) } + test("ALTER TABLE UNSET nonexistent property should throw an exception") { + val tableName = "test_table" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (a STRING, b INT) USING parquet") + + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER TABLE $tableName UNSET TBLPROPERTIES ('test_prop1', 'test_prop2', 'comment')") + }, + errorClass = "UNSET_NONEXISTENT_PROPERTIES", + parameters = Map( + "properties" -> "`test_prop1`, `test_prop2`", + "table" -> "`spark_catalog`.`default`.`test_table`") + ) + } + } + test("alter table: change column (datasource table)") { testChangeColumn(isDatasourceTable = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala index 6332c32c733a9..fc313de6c8fee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.errors.DataTypeErrors.{toSQLConf, toSQLStmt} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -388,6 +389,61 @@ class JDBCTableCatalogSuite extends QueryTest with SharedSparkSession { } } + test("CREATE NAMESPACE with LOCATION for JDBC catalog should throw an error") { + withTempDir { tempDir => + val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + Utils.classForName("org.h2.Driver") + withSQLConf( + "spark.sql.catalog.h2" -> classOf[JDBCTableCatalog].getName, + "spark.sql.catalog.h2.url" -> url, + "spark.sql.catalog.h2.driver" -> "org.h2.Driver") { + checkError( + exception = intercept[AnalysisException] { + sql("CREATE NAMESPACE h2.test_namespace LOCATION './samplepath'") + }, + errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND", + sqlState = "0A000", + parameters = Map("cmd" -> toSQLStmt("CREATE NAMESPACE ... LOCATION ..."))) + } + } + } + + test("ALTER NAMESPACE with property other than COMMENT " + + "for JDBC catalog should throw an exception") { + withTempDir { tempDir => + val url = s"jdbc:h2:${tempDir.getCanonicalPath};user=testUser;password=testPass" + Utils.classForName("org.h2.Driver") + withSQLConf( + "spark.sql.catalog.h2" -> classOf[JDBCTableCatalog].getName, + "spark.sql.catalog.h2.url" -> url, + "spark.sql.catalog.h2.driver" -> "org.h2.Driver") { + val namespace = "h2.test_namespace" + withNamespace(namespace) { + sql(s"CREATE NAMESPACE $namespace") + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER NAMESPACE h2.test_namespace SET LOCATION '/tmp/loc_test_2'") + }, + errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", + sqlState = "0A000", + parameters = Map( + "cmd" -> toSQLStmt("SET NAMESPACE"), + "property" -> toSQLConf("location"))) + + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER NAMESPACE h2.test_namespace SET PROPERTIES('a'='b')") + }, + errorClass = "NOT_SUPPORTED_IN_JDBC_CATALOG.COMMAND_WITH_PROPERTY", + sqlState = "0A000", + parameters = Map( + "cmd" -> toSQLStmt("SET NAMESPACE"), + "property" -> toSQLConf("a"))) + } + } + } + } + test("ALTER TABLE ... update column comment not supported") { val tableName = "h2.test.alt_table" withTable(tableName) { From ae518ecb7068347f70d947255eb54fdfd5ec8d48 Mon Sep 17 00:00:00 2001 From: Yousof Hosny Date: Mon, 11 Mar 2024 08:40:19 +0900 Subject: [PATCH 15/50] [SPARK-47218][SQL] XML: Changed SchemaOfXml to fail on DROPMALFORMED mode ### What changes were proposed in this pull request? Changed schema_of_xml should fail with an error on DROPMALFORMED mode to avoid creating schemas out of invalid XML. ### Why are the changes needed? DROPMALFORMED parse mode imply silently dropping the malformed record. But SchemaOfXml is expected to return a schema and may not have a valid schema to return for a malformed record. So DROPMALFORMED cannot be supported.. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45379 from yhosny/xml-parsemode-error. Authored-by: Yousof Hosny Signed-off-by: Hyukjin Kwon --- .../catalyst/expressions/xmlExpressions.scala | 8 +++-- .../execution/datasources/xml/XmlSuite.scala | 36 +++++++++++++++++++ 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala index 800515ca84b5e..8cc1c3a897457 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xmlExpressions.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.util.{ArrayData, FailFastMode, FailureSafeParser, GenericArrayData, PermissiveMode} +import org.apache.spark.sql.catalyst.util.{ArrayData, DropMalformedMode, FailFastMode, FailureSafeParser, GenericArrayData, PermissiveMode} import org.apache.spark.sql.catalyst.xml.{StaxXmlGenerator, StaxXmlParser, ValidatorUtil, XmlInferSchema, XmlOptions} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.internal.SQLConf @@ -189,8 +189,12 @@ case class SchemaOfXml( private lazy val xmlFactory = xmlOptions.buildXmlFactory() @transient - private lazy val xmlInferSchema = + private lazy val xmlInferSchema = { + if (xmlOptions.parseMode == DropMalformedMode) { + throw QueryCompilationErrors.parseModeUnsupportedError("schema_of_xml", xmlOptions.parseMode) + } new XmlInferSchema(xmlOptions, caseSensitive = SQLConf.get.caseSensitiveAnalysis) + } @transient private lazy val xml = child.eval().asInstanceOf[UTF8String] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala index 2194f76e7da6e..d7dc96184dab4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala @@ -1302,6 +1302,42 @@ class XmlSuite assert(result.select("decoded._corrupt_record").head().getString(0).nonEmpty) } + test("schema_of_xml with DROPMALFORMED parse error test") { + val e = intercept[AnalysisException] { + spark.sql(s"""SELECT schema_of_xml('1', map('mode', 'DROPMALFORMED'))""") + .collect() + } + checkError( + exception = e, + errorClass = "_LEGACY_ERROR_TEMP_1099", + parameters = Map( + "funcName" -> "schema_of_xml", + "mode" -> "DROPMALFORMED", + "permissiveMode" -> "PERMISSIVE", + "failFastMode" -> FailFastMode.name) + ) + } + + test("schema_of_xml with FAILFAST parse error test") { + val e = intercept[SparkException] { + spark.sql(s"""SELECT schema_of_xml('1', map('mode', 'FAILFAST'))""") + .collect() + } + checkError( + exception = e, + errorClass = "_LEGACY_ERROR_TEMP_2165", + parameters = Map( + "failFastMode" -> FailFastMode.name) + ) + } + + test("schema_of_xml with PERMISSIVE check no error test") { + val s = spark.sql(s"""SELECT schema_of_xml('1', map('mode', 'PERMISSIVE'))""") + .collect() + assert(s.head.get(0) == "STRUCT<_corrupt_record: STRING>") + } + + test("from_xml with PERMISSIVE parse mode with no corrupt col schema") { // XML contains error val xmlData = From edb970b8a73e5b1e08b01f9370dadb05a3e231e3 Mon Sep 17 00:00:00 2001 From: micheal-o Date: Mon, 11 Mar 2024 08:44:30 +0900 Subject: [PATCH 16/50] [SPARK-47200][SS] Make Foreach batch sink user function error handling backward compatible ### What changes were proposed in this pull request? I checked in a previous PR (https://github.com/apache/spark/pull/45299), that handles and classifies exceptions thrown in user provided functions for foreach batch sink. This change is to make it backward compatible in order not to break current users, since users may be depending on getting the user code error from the `StreamingQueryException.cause` instead of `StreamingQueryException.cause.cause` ### Why are the changes needed? To prevent breaking existing usage pattern. ### Does this PR introduce _any_ user-facing change? Yes, better error message with error class for ForeachBatchSink user function failures. ### How was this patch tested? updated existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #45449 from micheal-o/ForeachBatchExBackwardCompat. Authored-by: micheal-o Signed-off-by: Jungtaek Lim --- .../main/resources/error/error-classes.json | 2 +- docs/sql-error-conditions.md | 2 +- .../execution/streaming/StreamExecution.scala | 29 +++++++++++++------ .../streaming/sources/ForeachBatchSink.scala | 14 ++++++--- .../errors/QueryExecutionErrorsSuite.scala | 2 +- .../sources/ForeachBatchSinkSuite.scala | 17 ++++++----- 6 files changed, 43 insertions(+), 23 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 57746d6dbf1e9..9717ff2ed49c4 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1297,7 +1297,7 @@ }, "FOREACH_BATCH_USER_FUNCTION_ERROR" : { "message" : [ - "An error occurred in the user provided function in foreach batch sink." + "An error occurred in the user provided function in foreach batch sink. Reason: " ], "sqlState" : "39000" }, diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 7be01f8cb513d..0be75cde968fb 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -778,7 +778,7 @@ The operation `` is not allowed on the ``: `` [SQLSTATE: 39000](sql-error-conditions-sqlstates.html#class-39-external-routine-invocation-exception) -An error occurred in the user provided function in foreach batch sink. +An error occurred in the user provided function in foreach batch sink. Reason: `` ### FOUND_MULTIPLE_DATA_SOURCES diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 859fce8b1154c..50a73082a8c4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit, SparkDataStream} import org.apache.spark.sql.connector.write.{LogicalWriteInfoImpl, SupportsTruncate, Write} import org.apache.spark.sql.execution.command.StreamingExplainCommand +import org.apache.spark.sql.execution.streaming.sources.ForeachBatchUserFuncException import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.connector.SupportsStreamingUpdateAsAppend import org.apache.spark.sql.streaming._ @@ -279,6 +280,7 @@ abstract class StreamExecution( * `start()` method returns. */ private def runStream(): Unit = { + var errorClassOpt: Option[String] = None try { sparkSession.sparkContext.setJobGroup(runId.toString, getBatchDescriptionString, interruptOnCancel = true) @@ -330,9 +332,17 @@ abstract class StreamExecution( getLatestExecutionContext().updateStatusMessage("Stopped") case e: Throwable => val message = if (e.getMessage == null) "" else e.getMessage + val cause = if (e.isInstanceOf[ForeachBatchUserFuncException]) { + // We want to maintain the current way users get the causing exception + // from the StreamingQueryException. Hence the ForeachBatch exception is unwrapped here. + e.getCause + } else { + e + } + streamDeathCause = new StreamingQueryException( toDebugString(includeLogicalPlan = isInitialized), - cause = e, + cause = cause, getLatestExecutionContext().startOffsets .toOffsetSeq(sources.toSeq, getLatestExecutionContext().offsetSeqMetadata) .toString, @@ -350,12 +360,18 @@ abstract class StreamExecution( "endOffset" -> getLatestExecutionContext().endOffsets.toOffsetSeq( sources.toSeq, getLatestExecutionContext().offsetSeqMetadata).toString )) + + errorClassOpt = e match { + case t: SparkThrowable => Option(t.getErrorClass) + case _ => None + } + logError(s"Query $prettyIdString terminated with error", e) getLatestExecutionContext().updateStatusMessage(s"Terminated with exception: $message") // Rethrow the fatal errors to allow the user using `Thread.UncaughtExceptionHandler` to // handle them - if (!NonFatal(e)) { - throw e + if (!NonFatal(cause)) { + throw cause } } finally queryExecutionThread.runUninterruptibly { // The whole `finally` block must run inside `runUninterruptibly` to avoid being interrupted @@ -379,12 +395,6 @@ abstract class StreamExecution( // Notify others sparkSession.streams.notifyQueryTermination(StreamExecution.this) - val errorClassOpt = exception.flatMap { - _.cause match { - case t: SparkThrowable => Some(t.getErrorClass) - case _ => None - } - } postEvent( new QueryTerminatedEvent(id, runId, exception.map(_.cause).map(Utils.exceptionString), errorClassOpt)) @@ -691,6 +701,7 @@ object StreamExecution { case e2 @ (_: UncheckedIOException | _: ExecutionException | _: UncheckedExecutionException) if e2.getCause != null => isInterruptionException(e2.getCause, sc) + case fe: ForeachBatchUserFuncException => isInterruptionException(fe.getCause, sc) case se: SparkException => val jobGroup = sc.getLocalProperty("spark.jobGroup.id") if (jobGroup == null) return false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala index 5cf98dab21eb6..1262731790be1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSink.scala @@ -45,14 +45,20 @@ class ForeachBatchSink[T](batchWriter: (Dataset[T], Long) => Unit, encoder: Expr } catch { // The user code can throw any type of exception. case NonFatal(e) if !e.isInstanceOf[SparkThrowable] => - throw new SparkException( - errorClass = "FOREACH_BATCH_USER_FUNCTION_ERROR", - messageParameters = Map.empty, - cause = e) + throw ForeachBatchUserFuncException(e) } } } +/** + * Exception that wraps the exception thrown in the user provided function in ForeachBatch sink. + */ +private[streaming] case class ForeachBatchUserFuncException(cause: Throwable) + extends SparkException( + errorClass = "FOREACH_BATCH_USER_FUNCTION_ERROR", + messageParameters = Map("reason" -> Option(cause.getMessage).getOrElse("")), + cause = cause) + /** * Interface that is meant to be extended by Python classes via Py4J. * Py4J allows Python classes to implement Java interfaces so that the JVM can call back diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 9f7224315b0ef..d4697773742f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -889,7 +889,7 @@ class QueryExecutionErrorsSuite query.awaitTermination() } assert(e.getErrorClass === "STREAM_FAILED") - assert(e.getCause.getCause.isInstanceOf[NullPointerException]) + assert(e.getCause.isInstanceOf[NullPointerException]) } test("CONCURRENT_QUERY: streaming query is resumed from many sessions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala index 7d04d14a17abe..5304ea3b69ddc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming.sources import scala.collection.mutable import scala.language.implicitConversions -import org.apache.spark.{ExecutorDeadException, SparkException} +import org.apache.spark.ExecutorDeadException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.SerializeFromObjectExec @@ -193,23 +193,26 @@ class ForeachBatchSinkSuite extends StreamTest { mem.addData(1, 2, 3, 4, 5) val funcEx = new IllegalAccessException("access error") - val wrapperEx = intercept[StreamingQueryException] { + val queryEx = intercept[StreamingQueryException] { val query = ds.writeStream.foreachBatch((_: Dataset[Int], _: Long) => throw funcEx).start() query.awaitTermination() - }.getCause + } + + val errClass = "FOREACH_BATCH_USER_FUNCTION_ERROR" // verify that we classified the exception - checkError(wrapperEx.asInstanceOf[SparkException], "FOREACH_BATCH_USER_FUNCTION_ERROR") - assert(wrapperEx.getCause == funcEx) + assert(queryEx.getMessage.contains(errClass)) + assert(queryEx.getCause == funcEx) val sparkEx = ExecutorDeadException("network error") val ex = intercept[StreamingQueryException] { val query = ds.writeStream.foreachBatch((_: Dataset[Int], _: Long) => throw sparkEx).start() query.awaitTermination() - }.getCause + } // we didn't wrap the spark exception - assert(ex == sparkEx) + assert(!ex.getMessage.contains(errClass)) + assert(ex.getCause == sparkEx) } // ============== Helper classes and methods ================= From f6a00f2dda9379a3b91297a556953d6f4c0f84cd Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 11 Mar 2024 08:53:36 +0900 Subject: [PATCH 17/50] [SPARK-47325][INFRA] Use the latest `buf-setup-action` in github workflow ### What changes were proposed in this pull request? The pr aims to `unpin` specific version `buf-setup-action` in github workflow building. ### Why are the changes needed? - [The last](https://github.com/apache/spark/pull/45205) `pin` to a `specific version` was due to a bug in the version `v1.29.0-1`. The latest version has been upgraded to `v1.30.0`, and testing has found that this version is ok. - This latest version `v1.30.0` has a `change` regarding the upgrade from `node16` to `node20`. https://github.com/bufbuild/buf-setup-action/compare/v1.29.0...v1.30.0 image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45433 from panbingkun/test_buf-setup-action_1_30_0. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index a24309e137eb8..4f2be1c04f988 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -574,9 +574,8 @@ jobs: 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 - name: Install Buf - uses: bufbuild/buf-setup-action@v1.29.0 + uses: bufbuild/buf-setup-action@v1 with: - version: 1.29.0 github_token: ${{ secrets.GITHUB_TOKEN }} - name: Protocol Buffers Linter uses: bufbuild/buf-lint-action@v1 From a4603f134fb0d496109d4c90889191c506e82691 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 11 Mar 2024 09:06:20 +0900 Subject: [PATCH 18/50] [SPARK-47334][SQL] Make `withColumnRenamed` reuse the implementation of `withColumnsRenamed` ### What changes were proposed in this pull request? Make `withColumnRenamed` reuse the implementation of `withColumnsRenamed` ### Why are the changes needed? to avoid any divergence in the future ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #45450 from zhengruifeng/with_rename_consistent. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/Dataset.scala | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 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 f3bf6119659d1..f0c9f7ae53fc4 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 @@ -2876,23 +2876,8 @@ class Dataset[T] private[sql]( * @group untypedrel * @since 2.0.0 */ - def withColumnRenamed(existingName: String, newName: String): DataFrame = { - val resolver = sparkSession.sessionState.analyzer.resolver - val output = queryExecution.analyzed.output - val shouldRename = output.exists(f => resolver(f.name, existingName)) - if (shouldRename) { - val columns = output.map { col => - if (resolver(col.name, existingName)) { - Column(col).as(newName) - } else { - Column(col) - } - } - select(columns : _*) - } else { - toDF() - } - } + def withColumnRenamed(existingName: String, newName: String): DataFrame = + withColumnsRenamed(Seq(existingName), Seq(newName)) /** * (Scala-specific) @@ -2921,18 +2906,24 @@ class Dataset[T] private[sql]( val resolver = sparkSession.sessionState.analyzer.resolver val output: Seq[NamedExpression] = queryExecution.analyzed.output + var shouldRename = false val projectList = colNames.zip(newColNames).foldLeft(output) { case (attrs, (existingName, newName)) => attrs.map(attr => if (resolver(attr.name, existingName)) { + shouldRename = true Alias(attr, newName)() } else { attr } ) } - withPlan(Project(projectList, logicalPlan)) + if (shouldRename) { + withPlan(Project(projectList, logicalPlan)) + } else { + toDF() + } } /** From f6df78154bac826bd51d2aad185ce02a7efd36b6 Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Mon, 11 Mar 2024 09:07:35 +0900 Subject: [PATCH 19/50] [SPARK-47035][SS][CONNECT][FOLLOWUP] Additional protobuf change to client side listener ### What changes were proposed in this pull request? Followup of previous protocol change https://github.com/apache/spark/pull/45091. Add the request proto `Command` and response proto message to `ExecutePlanResponse` ### Why are the changes needed? Continuation of client side listener for spark connect ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Will be tested in subsequent PR, the proto change itself doesn't do any harm ### Was this patch authored or co-authored using generative AI tooling? No Closes #45444 from WweiL/SPARK-47035-protocol-followup. Authored-by: Wei Liu Signed-off-by: Hyukjin Kwon --- .../main/protobuf/spark/connect/base.proto | 3 + .../protobuf/spark/connect/commands.proto | 2 + python/pyspark/sql/connect/proto/base_pb2.py | 204 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 13 ++ .../pyspark/sql/connect/proto/commands_pb2.py | 180 ++++++++-------- .../sql/connect/proto/commands_pb2.pyi | 40 +++- 6 files changed, 249 insertions(+), 193 deletions(-) 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 f24ca0a8fc3b1..cb9dbe62c193b 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -357,6 +357,9 @@ message ExecutePlanResponse { // Response for commands on the streaming query manager. StreamingQueryManagerCommandResult streaming_query_manager_command_result = 11; + // Response for commands on the client side streaming query listener. + StreamingQueryListenerEventsResult streaming_query_listener_events_result = 16; + // Response type informing if the stream is complete in reattachable execution. ResultComplete result_complete = 14; diff --git a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto index e845d5f290618..76ac106b1de81 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -42,6 +42,7 @@ message Command { GetResourcesCommand get_resources_command = 8; StreamingQueryManagerCommand streaming_query_manager_command = 9; CommonInlineUserDefinedTableFunction register_table_function = 10; + StreamingQueryListenerBusCommand streaming_query_listener_bus_command = 11; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // Commands they can add them here. During the planning the correct resolution is done. @@ -456,6 +457,7 @@ message StreamingQueryListenerEvent { message StreamingQueryListenerEventsResult { repeated StreamingQueryListenerEvent events = 1; + optional bool listener_bus_listener_added = 2; } // Command to get the output of 'SparkContext.resources' diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 8326ce511d56a..1941900ae69d2 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"\xf5\x12\n\x12\x41nalyzePlanRequest\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\x01R\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\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"\xa0\x04\n\x12\x45xecutePlanRequest\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\x0coperation_id\x18\x06 \x01(\tH\x00R\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\x01R\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\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\x9b\x10\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\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\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\x0eResultCompleteB\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"\x84\x08\n\rConfigRequest\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\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\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\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"\xe7\x06\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\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x01R\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\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"\xc3\x01\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB\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"\xd8\x03\n\x10InterruptRequest\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\x01R\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\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"\x93\x02\n\x16ReattachExecuteRequest\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\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x01R\x0elastResponseId\x88\x01\x01\x42\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc6\x03\n\x15ReleaseExecuteRequest\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\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\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\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"\xc9\x01\n\x18\x46\x65tchErrorDetailsRequest\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\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\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"\xf5\x12\n\x12\x41nalyzePlanRequest\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\x01R\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\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"\xa0\x04\n\x12\x45xecutePlanRequest\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\x0coperation_id\x18\x06 \x01(\tH\x00R\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\x01R\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\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xa5\x11\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\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\x0eResultCompleteB\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"\x84\x08\n\rConfigRequest\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\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\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\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"\xe7\x06\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\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x01R\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\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"\xc3\x01\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB\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"\xd8\x03\n\x10InterruptRequest\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\x01R\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\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"\x93\x02\n\x16ReattachExecuteRequest\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\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x01R\x0elastResponseId\x88\x01\x01\x42\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc6\x03\n\x15ReleaseExecuteRequest\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\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\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\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"\xc9\x01\n\x18\x46\x65tchErrorDetailsRequest\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\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\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,105 +120,105 @@ _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_start = 4977 _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_end = 5142 _EXECUTEPLANRESPONSE._serialized_start = 5178 - _EXECUTEPLANRESPONSE._serialized_end = 7253 - _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_start = 6389 - _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_end = 6460 - _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 6462 - _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_end = 6580 - _EXECUTEPLANRESPONSE_METRICS._serialized_start = 6583 - _EXECUTEPLANRESPONSE_METRICS._serialized_end = 7100 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_start = 6678 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_end = 7010 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_start = 6887 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 7010 - _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 7012 - _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 7100 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7102 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7218 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7220 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7236 - _KEYVALUE._serialized_start = 7255 - _KEYVALUE._serialized_end = 7320 - _CONFIGREQUEST._serialized_start = 7323 - _CONFIGREQUEST._serialized_end = 8351 - _CONFIGREQUEST_OPERATION._serialized_start = 7543 - _CONFIGREQUEST_OPERATION._serialized_end = 8041 - _CONFIGREQUEST_SET._serialized_start = 8043 - _CONFIGREQUEST_SET._serialized_end = 8095 - _CONFIGREQUEST_GET._serialized_start = 8097 - _CONFIGREQUEST_GET._serialized_end = 8122 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 8124 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 8187 - _CONFIGREQUEST_GETOPTION._serialized_start = 8189 - _CONFIGREQUEST_GETOPTION._serialized_end = 8220 - _CONFIGREQUEST_GETALL._serialized_start = 8222 - _CONFIGREQUEST_GETALL._serialized_end = 8270 - _CONFIGREQUEST_UNSET._serialized_start = 8272 - _CONFIGREQUEST_UNSET._serialized_end = 8299 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 8301 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 8335 - _CONFIGRESPONSE._serialized_start = 8354 - _CONFIGRESPONSE._serialized_end = 8529 - _ADDARTIFACTSREQUEST._serialized_start = 8532 - _ADDARTIFACTSREQUEST._serialized_end = 9403 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 8919 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 8972 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 8974 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 9085 - _ADDARTIFACTSREQUEST_BATCH._serialized_start = 9087 - _ADDARTIFACTSREQUEST_BATCH._serialized_end = 9180 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 9183 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 9376 - _ADDARTIFACTSRESPONSE._serialized_start = 9406 - _ADDARTIFACTSRESPONSE._serialized_end = 9678 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 9597 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 9678 - _ARTIFACTSTATUSESREQUEST._serialized_start = 9681 - _ARTIFACTSTATUSESREQUEST._serialized_end = 9876 - _ARTIFACTSTATUSESRESPONSE._serialized_start = 9879 - _ARTIFACTSTATUSESRESPONSE._serialized_end = 10231 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 10074 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 10189 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 10191 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 10231 - _INTERRUPTREQUEST._serialized_start = 10234 - _INTERRUPTREQUEST._serialized_end = 10706 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 10549 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 10677 - _INTERRUPTRESPONSE._serialized_start = 10709 - _INTERRUPTRESPONSE._serialized_end = 10853 - _REATTACHOPTIONS._serialized_start = 10855 - _REATTACHOPTIONS._serialized_end = 10908 - _REATTACHEXECUTEREQUEST._serialized_start = 10911 - _REATTACHEXECUTEREQUEST._serialized_end = 11186 - _RELEASEEXECUTEREQUEST._serialized_start = 11189 - _RELEASEEXECUTEREQUEST._serialized_end = 11643 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 11555 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 11567 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 11569 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 11616 - _RELEASEEXECUTERESPONSE._serialized_start = 11646 - _RELEASEEXECUTERESPONSE._serialized_end = 11811 - _RELEASESESSIONREQUEST._serialized_start = 11814 - _RELEASESESSIONREQUEST._serialized_end = 11985 - _RELEASESESSIONRESPONSE._serialized_start = 11987 - _RELEASESESSIONRESPONSE._serialized_end = 12095 - _FETCHERRORDETAILSREQUEST._serialized_start = 12098 - _FETCHERRORDETAILSREQUEST._serialized_end = 12299 - _FETCHERRORDETAILSRESPONSE._serialized_start = 12302 - _FETCHERRORDETAILSRESPONSE._serialized_end = 13857 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 12531 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 12705 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 12708 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 13076 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 13039 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 13076 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 13079 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 13488 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 13390 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 13458 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 13491 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 13838 - _SPARKCONNECTSERVICE._serialized_start = 13860 - _SPARKCONNECTSERVICE._serialized_end = 14806 + _EXECUTEPLANRESPONSE._serialized_end = 7391 + _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_start = 6527 + _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_end = 6598 + _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 6600 + _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_end = 6718 + _EXECUTEPLANRESPONSE_METRICS._serialized_start = 6721 + _EXECUTEPLANRESPONSE_METRICS._serialized_end = 7238 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_start = 6816 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_end = 7148 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_start = 7025 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 7148 + _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 7150 + _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 7238 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7240 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7356 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7358 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7374 + _KEYVALUE._serialized_start = 7393 + _KEYVALUE._serialized_end = 7458 + _CONFIGREQUEST._serialized_start = 7461 + _CONFIGREQUEST._serialized_end = 8489 + _CONFIGREQUEST_OPERATION._serialized_start = 7681 + _CONFIGREQUEST_OPERATION._serialized_end = 8179 + _CONFIGREQUEST_SET._serialized_start = 8181 + _CONFIGREQUEST_SET._serialized_end = 8233 + _CONFIGREQUEST_GET._serialized_start = 8235 + _CONFIGREQUEST_GET._serialized_end = 8260 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 8262 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 8325 + _CONFIGREQUEST_GETOPTION._serialized_start = 8327 + _CONFIGREQUEST_GETOPTION._serialized_end = 8358 + _CONFIGREQUEST_GETALL._serialized_start = 8360 + _CONFIGREQUEST_GETALL._serialized_end = 8408 + _CONFIGREQUEST_UNSET._serialized_start = 8410 + _CONFIGREQUEST_UNSET._serialized_end = 8437 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 8439 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 8473 + _CONFIGRESPONSE._serialized_start = 8492 + _CONFIGRESPONSE._serialized_end = 8667 + _ADDARTIFACTSREQUEST._serialized_start = 8670 + _ADDARTIFACTSREQUEST._serialized_end = 9541 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 9057 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 9110 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 9112 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 9223 + _ADDARTIFACTSREQUEST_BATCH._serialized_start = 9225 + _ADDARTIFACTSREQUEST_BATCH._serialized_end = 9318 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 9321 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 9514 + _ADDARTIFACTSRESPONSE._serialized_start = 9544 + _ADDARTIFACTSRESPONSE._serialized_end = 9816 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 9735 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 9816 + _ARTIFACTSTATUSESREQUEST._serialized_start = 9819 + _ARTIFACTSTATUSESREQUEST._serialized_end = 10014 + _ARTIFACTSTATUSESRESPONSE._serialized_start = 10017 + _ARTIFACTSTATUSESRESPONSE._serialized_end = 10369 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 10212 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 10327 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 10329 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 10369 + _INTERRUPTREQUEST._serialized_start = 10372 + _INTERRUPTREQUEST._serialized_end = 10844 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 10687 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 10815 + _INTERRUPTRESPONSE._serialized_start = 10847 + _INTERRUPTRESPONSE._serialized_end = 10991 + _REATTACHOPTIONS._serialized_start = 10993 + _REATTACHOPTIONS._serialized_end = 11046 + _REATTACHEXECUTEREQUEST._serialized_start = 11049 + _REATTACHEXECUTEREQUEST._serialized_end = 11324 + _RELEASEEXECUTEREQUEST._serialized_start = 11327 + _RELEASEEXECUTEREQUEST._serialized_end = 11781 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 11693 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 11705 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 11707 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 11754 + _RELEASEEXECUTERESPONSE._serialized_start = 11784 + _RELEASEEXECUTERESPONSE._serialized_end = 11949 + _RELEASESESSIONREQUEST._serialized_start = 11952 + _RELEASESESSIONREQUEST._serialized_end = 12123 + _RELEASESESSIONRESPONSE._serialized_start = 12125 + _RELEASESESSIONRESPONSE._serialized_end = 12233 + _FETCHERRORDETAILSREQUEST._serialized_start = 12236 + _FETCHERRORDETAILSREQUEST._serialized_end = 12437 + _FETCHERRORDETAILSRESPONSE._serialized_start = 12440 + _FETCHERRORDETAILSRESPONSE._serialized_end = 13995 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 12669 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 12843 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 12846 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 13214 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 13177 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 13214 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 13217 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 13626 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 13528 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 13596 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 13629 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 13976 + _SPARKCONNECTSERVICE._serialized_start = 13998 + _SPARKCONNECTSERVICE._serialized_end = 14944 # @@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 e4ed03dc6945b..5ed2d207aca5f 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -1410,6 +1410,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): STREAMING_QUERY_COMMAND_RESULT_FIELD_NUMBER: builtins.int GET_RESOURCES_COMMAND_RESULT_FIELD_NUMBER: builtins.int STREAMING_QUERY_MANAGER_COMMAND_RESULT_FIELD_NUMBER: builtins.int + STREAMING_QUERY_LISTENER_EVENTS_RESULT_FIELD_NUMBER: builtins.int RESULT_COMPLETE_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int METRICS_FIELD_NUMBER: builtins.int @@ -1456,6 +1457,11 @@ class ExecutePlanResponse(google.protobuf.message.Message): ) -> pyspark.sql.connect.proto.commands_pb2.StreamingQueryManagerCommandResult: """Response for commands on the streaming query manager.""" @property + def streaming_query_listener_events_result( + self, + ) -> pyspark.sql.connect.proto.commands_pb2.StreamingQueryListenerEventsResult: + """Response for commands on the client side streaming query listener.""" + @property def result_complete(self) -> global___ExecutePlanResponse.ResultComplete: """Response type informing if the stream is complete in reattachable execution.""" @property @@ -1493,6 +1499,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): | None = ..., streaming_query_manager_command_result: pyspark.sql.connect.proto.commands_pb2.StreamingQueryManagerCommandResult | None = ..., + streaming_query_listener_events_result: pyspark.sql.connect.proto.commands_pb2.StreamingQueryListenerEventsResult + | None = ..., result_complete: global___ExecutePlanResponse.ResultComplete | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., metrics: global___ExecutePlanResponse.Metrics | None = ..., @@ -1521,6 +1529,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"sql_command_result", "streaming_query_command_result", b"streaming_query_command_result", + "streaming_query_listener_events_result", + b"streaming_query_listener_events_result", "streaming_query_manager_command_result", b"streaming_query_manager_command_result", "write_stream_operation_start_result", @@ -1558,6 +1568,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"sql_command_result", "streaming_query_command_result", b"streaming_query_command_result", + "streaming_query_listener_events_result", + b"streaming_query_listener_events_result", "streaming_query_manager_command_result", b"streaming_query_manager_command_result", "write_stream_operation_start_result", @@ -1574,6 +1586,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): "streaming_query_command_result", "get_resources_command_result", "streaming_query_manager_command_result", + "streaming_query_listener_events_result", "result_complete", "extension", ] diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index cfeb57a0c56a8..45e276bf96e76 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xf5\x07\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xef\x03\n\nSqlCommand\x12\x10\n\x03sql\x18\x01 \x01(\tR\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12V\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"h\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xf9\x08\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xef\x03\n\nSqlCommand\x12\x10\n\x03sql\x18\x01 \x01(\tR\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12V\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -63,94 +63,94 @@ _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_options = b"8\001" _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._options = None _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_options = b"8\001" - _STREAMINGQUERYEVENTTYPE._serialized_start = 9409 - _STREAMINGQUERYEVENTTYPE._serialized_end = 9542 + _STREAMINGQUERYEVENTTYPE._serialized_start = 9642 + _STREAMINGQUERYEVENTTYPE._serialized_end = 9775 _COMMAND._serialized_start = 167 - _COMMAND._serialized_end = 1180 - _SQLCOMMAND._serialized_start = 1183 - _SQLCOMMAND._serialized_end = 1678 - _SQLCOMMAND_ARGSENTRY._serialized_start = 1494 - _SQLCOMMAND_ARGSENTRY._serialized_end = 1584 - _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_start = 1586 - _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_end = 1678 - _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 1681 - _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 1831 - _WRITEOPERATION._serialized_start = 1834 - _WRITEOPERATION._serialized_end = 2932 - _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2356 - _WRITEOPERATION_OPTIONSENTRY._serialized_end = 2414 - _WRITEOPERATION_SAVETABLE._serialized_start = 2417 - _WRITEOPERATION_SAVETABLE._serialized_end = 2675 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 2551 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 2675 - _WRITEOPERATION_BUCKETBY._serialized_start = 2677 - _WRITEOPERATION_BUCKETBY._serialized_end = 2768 - _WRITEOPERATION_SAVEMODE._serialized_start = 2771 - _WRITEOPERATION_SAVEMODE._serialized_end = 2908 - _WRITEOPERATIONV2._serialized_start = 2935 - _WRITEOPERATIONV2._serialized_end = 3795 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2356 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 2414 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 3554 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 3620 - _WRITEOPERATIONV2_MODE._serialized_start = 3623 - _WRITEOPERATIONV2_MODE._serialized_end = 3782 - _WRITESTREAMOPERATIONSTART._serialized_start = 3798 - _WRITESTREAMOPERATIONSTART._serialized_end = 4598 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2356 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 2414 - _STREAMINGFOREACHFUNCTION._serialized_start = 4601 - _STREAMINGFOREACHFUNCTION._serialized_end = 4780 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 4783 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 4995 - _STREAMINGQUERYINSTANCEID._serialized_start = 4997 - _STREAMINGQUERYINSTANCEID._serialized_end = 5062 - _STREAMINGQUERYCOMMAND._serialized_start = 5065 - _STREAMINGQUERYCOMMAND._serialized_end = 5697 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 5564 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 5608 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 5610 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 5686 - _STREAMINGQUERYCOMMANDRESULT._serialized_start = 5700 - _STREAMINGQUERYCOMMANDRESULT._serialized_end = 6841 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6283 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 6453 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 6455 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 6527 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 6529 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 6568 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 6571 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 6768 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 6770 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 6826 - _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 6844 - _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 7673 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 7375 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 7454 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 7457 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 7662 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 7676 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 8752 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8284 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 8411 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 8413 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 8528 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 8530 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 8589 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 8591 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 8666 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 8668 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 8737 - _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_start = 8755 - _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_end = 8928 - _STREAMINGQUERYLISTENEREVENT._serialized_start = 8931 - _STREAMINGQUERYLISTENEREVENT._serialized_end = 9062 - _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_start = 9064 - _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_end = 9168 - _GETRESOURCESCOMMAND._serialized_start = 9170 - _GETRESOURCESCOMMAND._serialized_end = 9191 - _GETRESOURCESCOMMANDRESULT._serialized_start = 9194 - _GETRESOURCESCOMMANDRESULT._serialized_end = 9406 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 9310 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 9406 + _COMMAND._serialized_end = 1312 + _SQLCOMMAND._serialized_start = 1315 + _SQLCOMMAND._serialized_end = 1810 + _SQLCOMMAND_ARGSENTRY._serialized_start = 1626 + _SQLCOMMAND_ARGSENTRY._serialized_end = 1716 + _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_start = 1718 + _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_end = 1810 + _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 1813 + _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 1963 + _WRITEOPERATION._serialized_start = 1966 + _WRITEOPERATION._serialized_end = 3064 + _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2488 + _WRITEOPERATION_OPTIONSENTRY._serialized_end = 2546 + _WRITEOPERATION_SAVETABLE._serialized_start = 2549 + _WRITEOPERATION_SAVETABLE._serialized_end = 2807 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 2683 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 2807 + _WRITEOPERATION_BUCKETBY._serialized_start = 2809 + _WRITEOPERATION_BUCKETBY._serialized_end = 2900 + _WRITEOPERATION_SAVEMODE._serialized_start = 2903 + _WRITEOPERATION_SAVEMODE._serialized_end = 3040 + _WRITEOPERATIONV2._serialized_start = 3067 + _WRITEOPERATIONV2._serialized_end = 3927 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2488 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 2546 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 3686 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 3752 + _WRITEOPERATIONV2_MODE._serialized_start = 3755 + _WRITEOPERATIONV2_MODE._serialized_end = 3914 + _WRITESTREAMOPERATIONSTART._serialized_start = 3930 + _WRITESTREAMOPERATIONSTART._serialized_end = 4730 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2488 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 2546 + _STREAMINGFOREACHFUNCTION._serialized_start = 4733 + _STREAMINGFOREACHFUNCTION._serialized_end = 4912 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 4915 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 5127 + _STREAMINGQUERYINSTANCEID._serialized_start = 5129 + _STREAMINGQUERYINSTANCEID._serialized_end = 5194 + _STREAMINGQUERYCOMMAND._serialized_start = 5197 + _STREAMINGQUERYCOMMAND._serialized_end = 5829 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 5696 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 5740 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 5742 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 5818 + _STREAMINGQUERYCOMMANDRESULT._serialized_start = 5832 + _STREAMINGQUERYCOMMANDRESULT._serialized_end = 6973 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6415 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 6585 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 6587 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 6659 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 6661 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 6700 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 6703 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 6900 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 6902 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 6958 + _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 6976 + _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 7805 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 7507 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 7586 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 7589 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 7794 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 7808 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 8884 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8416 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 8543 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 8545 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 8660 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 8662 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 8721 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 8723 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 8798 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 8800 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 8869 + _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_start = 8887 + _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_end = 9060 + _STREAMINGQUERYLISTENEREVENT._serialized_start = 9063 + _STREAMINGQUERYLISTENEREVENT._serialized_end = 9194 + _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_start = 9197 + _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_end = 9401 + _GETRESOURCESCOMMAND._serialized_start = 9403 + _GETRESOURCESCOMMAND._serialized_end = 9424 + _GETRESOURCESCOMMANDRESULT._serialized_start = 9427 + _GETRESOURCESCOMMANDRESULT._serialized_end = 9639 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 9543 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 9639 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index e5b09bf0b2860..fbbec4f81d2d8 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -98,6 +98,7 @@ class Command(google.protobuf.message.Message): GET_RESOURCES_COMMAND_FIELD_NUMBER: builtins.int STREAMING_QUERY_MANAGER_COMMAND_FIELD_NUMBER: builtins.int REGISTER_TABLE_FUNCTION_FIELD_NUMBER: builtins.int + STREAMING_QUERY_LISTENER_BUS_COMMAND_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def register_function( @@ -124,6 +125,8 @@ class Command(google.protobuf.message.Message): self, ) -> pyspark.sql.connect.proto.relations_pb2.CommonInlineUserDefinedTableFunction: ... @property + def streaming_query_listener_bus_command(self) -> global___StreamingQueryListenerBusCommand: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary Commands they can add them here. During the planning the correct resolution is done. @@ -143,6 +146,8 @@ class Command(google.protobuf.message.Message): streaming_query_manager_command: global___StreamingQueryManagerCommand | None = ..., register_table_function: pyspark.sql.connect.proto.relations_pb2.CommonInlineUserDefinedTableFunction | None = ..., + streaming_query_listener_bus_command: global___StreamingQueryListenerBusCommand + | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -164,6 +169,8 @@ class Command(google.protobuf.message.Message): b"sql_command", "streaming_query_command", b"streaming_query_command", + "streaming_query_listener_bus_command", + b"streaming_query_listener_bus_command", "streaming_query_manager_command", b"streaming_query_manager_command", "write_operation", @@ -193,6 +200,8 @@ class Command(google.protobuf.message.Message): b"sql_command", "streaming_query_command", b"streaming_query_command", + "streaming_query_listener_bus_command", + b"streaming_query_listener_bus_command", "streaming_query_manager_command", b"streaming_query_manager_command", "write_operation", @@ -217,6 +226,7 @@ class Command(google.protobuf.message.Message): "get_resources_command", "streaming_query_manager_command", "register_table_function", + "streaming_query_listener_bus_command", "extension", ] | None @@ -1938,18 +1948,46 @@ class StreamingQueryListenerEventsResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor EVENTS_FIELD_NUMBER: builtins.int + LISTENER_BUS_LISTENER_ADDED_FIELD_NUMBER: builtins.int @property def events( self, ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ global___StreamingQueryListenerEvent ]: ... + listener_bus_listener_added: builtins.bool def __init__( self, *, events: collections.abc.Iterable[global___StreamingQueryListenerEvent] | None = ..., + listener_bus_listener_added: builtins.bool | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_listener_bus_listener_added", + b"_listener_bus_listener_added", + "listener_bus_listener_added", + b"listener_bus_listener_added", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_listener_bus_listener_added", + b"_listener_bus_listener_added", + "events", + b"events", + "listener_bus_listener_added", + b"listener_bus_listener_added", + ], ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["events", b"events"]) -> None: ... + def WhichOneof( + self, + oneof_group: typing_extensions.Literal[ + "_listener_bus_listener_added", b"_listener_bus_listener_added" + ], + ) -> typing_extensions.Literal["listener_bus_listener_added"] | None: ... global___StreamingQueryListenerEventsResult = StreamingQueryListenerEventsResult From 5ac560c76e6027cfda6893fe1ea61e0bedffccbe Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Mon, 11 Mar 2024 09:20:35 +0900 Subject: [PATCH 20/50] [MINOR][DOCS][SQL] Fix doc comment for coalescePartitions.parallelismFirst This changes the second `true` to `false` to make the doc comment correct. A setting of false will mean to not prioritize parallism and that will lead to less small tasks. Seems like it incorrectness was introduced here: https://github.com/apache/spark/pull/44787 ### What changes were proposed in this pull request? Documentation fix. ### Why are the changes needed? Current doc is wrong. ### 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 #45437 from eejbyfeldt/fix-minor-doc-comment-on-parallism-first. Authored-by: Emil Ejbyfeldt Signed-off-by: Hyukjin Kwon --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4ca5300239223..761b0ea72f3db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -710,7 +710,7 @@ object SQLConf { "parallelism of the Spark cluster. The calculated size is usually smaller than the " + "configured target size. This is to maximize the parallelism and avoid performance " + "regressions when enabling adaptive query execution. It's recommended to set this " + - "config to true on a busy cluster to make resource utilization more efficient (not many " + + "config to false on a busy cluster to make resource utilization more efficient (not many " + "small tasks).") .version("3.2.0") .booleanConf From afbebfbadc4b5e927df7c568a8afb08fc4407f58 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 11 Mar 2024 09:20:44 +0900 Subject: [PATCH 21/50] [SPARK-47331][SS] Serialization using case classes/primitives/POJO based on SQL encoder for Arbitrary State API v2 ### What changes were proposed in this pull request? In the new operator for arbitrary state-v2, we cannot rely on the session/encoder being available since the initialization for the various state instances happens on the executors. Hence, for the state serialization, we propose to let user explicitly pass in encoder for state variable and serialize primitives/case classes/POJO with SQL encoder. Leveraging SQL encoder can speed up the serialization. ### Why are the changes needed? These changes are needed for providing a dedicated serializer for state-v2. The changes are part of the work around adding new stateful streaming operator for arbitrary state mgmt that provides a bunch of new features listed in the SPIP JIRA here - https://issues.apache.org/jira/browse/SPARK-45939 ### Does this PR introduce _any_ user-facing change? Users will need to specify the SQL encoder for their state variable: `def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T]` `def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T]` For primitive type, Encoder is something as: `Encoders.scalaLong`; for case class, `Encoders.product[CaseClass]`; for POJO, `Encoders.bean(classOf[POJOClass])` ### How was this patch tested? Unit tests for primitives, case classes, POJO separately in `ValueStateSuite` ### Was this patch authored or co-authored using generative AI tooling? No Closes #45447 from jingz-db/sql-encoder-state-v2. Authored-by: jingz-db Signed-off-by: Jungtaek Lim --- .../streaming/StatefulProcessorHandle.scala | 7 +- .../execution/streaming/ListStateImpl.scala | 8 +- .../streaming/StateTypesEncoderUtils.scala | 41 +++--- .../StatefulProcessorHandleImpl.scala | 9 +- .../execution/streaming/ValueStateImpl.scala | 9 +- .../streaming/state/POJOTestClass.java | 78 ++++++++++++ .../streaming/state/ValueStateSuite.scala | 117 +++++++++++++++++- .../TransformWithListStateSuite.scala | 7 +- .../streaming/TransformWithStateSuite.scala | 11 +- 9 files changed, 250 insertions(+), 37 deletions(-) create mode 100644 sql/core/src/test/java/org/apache/spark/sql/execution/streaming/state/POJOTestClass.java diff --git a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala index 5d3390f80f6d0..86bf1e85f90cf 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming import java.io.Serializable import org.apache.spark.annotation.{Evolving, Experimental} +import org.apache.spark.sql.Encoder /** * Represents the operation handle provided to the stateful processor used in the @@ -33,20 +34,22 @@ private[sql] trait StatefulProcessorHandle extends Serializable { * The user must ensure to call this function only within the `init()` method of the * StatefulProcessor. * @param stateName - name of the state variable + * @param valEncoder - SQL encoder for state variable * @tparam T - type of state variable * @return - instance of ValueState of type T that can be used to store state persistently */ - def getValueState[T](stateName: String): ValueState[T] + def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] /** * Creates new or returns existing list state associated with stateName. * The ListState persists values of type T. * * @param stateName - name of the state variable + * @param valEncoder - SQL encoder for state variable * @tparam T - type of state variable * @return - instance of ListState of type T that can be used to store state persistently */ - def getListState[T](stateName: String): ListState[T] + def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T] /** Function to return queryInfo for currently running task */ def getQueryInfo(): QueryInfo diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala index b6ed48dab579a..d0be62293d058 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging +import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA} import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreErrors} @@ -28,17 +29,20 @@ import org.apache.spark.sql.streaming.ListState * * @param store - reference to the StateStore instance to be used for storing state * @param stateName - name of logical state partition + * @param keyEnc - Spark SQL encoder for key + * @param valEncoder - Spark SQL encoder for value * @tparam S - data type of object that will be stored in the list */ class ListStateImpl[S]( store: StateStore, stateName: String, - keyExprEnc: ExpressionEncoder[Any]) + keyExprEnc: ExpressionEncoder[Any], + valEncoder: Encoder[S]) extends ListState[S] with Logging { private val keySerializer = keyExprEnc.createSerializer() - private val stateTypesEncoder = StateTypesEncoder(keySerializer, stateName) + private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder, stateName) store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, numColsPrefixKey = 0, VALUE_ROW_SCHEMA, useMultipleValuesPerKey = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala index 15d77030d57b1..36758eafa3928 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.execution.streaming -import org.apache.commons.lang3.SerializationUtils - +import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.Serializer +import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.streaming.state.StateStoreErrors import org.apache.spark.sql.types.{BinaryType, StructType} @@ -41,17 +41,27 @@ object StateKeyValueRowSchema { * * @param keySerializer - serializer to serialize the grouping key of type `GK` * to an [[InternalRow]] + * @param valEncoder - SQL encoder for value of type `S` * @param stateName - name of logical state partition * @tparam GK - grouping key type + * @tparam V - value type */ -class StateTypesEncoder[GK]( +class StateTypesEncoder[GK, V]( keySerializer: Serializer[GK], + valEncoder: Encoder[V], stateName: String) { import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._ + /** Variables reused for conversions between byte array and UnsafeRow */ private val keyProjection = UnsafeProjection.create(KEY_ROW_SCHEMA) private val valueProjection = UnsafeProjection.create(VALUE_ROW_SCHEMA) + /** Variables reused for value conversions between spark sql and object */ + private val valExpressionEnc = encoderFor(valEncoder) + private val objToRowSerializer = valExpressionEnc.createSerializer() + private val rowToObjDeserializer = valExpressionEnc.resolveAndBind().createDeserializer() + private val reuseRow = new UnsafeRow(valEncoder.schema.fields.length) + // TODO: validate places that are trying to encode the key and check if we can eliminate/ // add caching for some of these calls. def encodeGroupingKey(): UnsafeRow = { @@ -66,23 +76,26 @@ class StateTypesEncoder[GK]( keyRow } - def encodeValue[S](value: S): UnsafeRow = { - val valueByteArr = SerializationUtils.serialize(value.asInstanceOf[Serializable]) - val valueRow = valueProjection(InternalRow(valueByteArr)) - valueRow + def encodeValue(value: V): UnsafeRow = { + val objRow: InternalRow = objToRowSerializer.apply(value) + val bytes = objRow.asInstanceOf[UnsafeRow].getBytes() + val valRow = valueProjection(InternalRow(bytes)) + valRow } - def decodeValue[S](row: UnsafeRow): S = { - SerializationUtils - .deserialize(row.getBinary(0)) - .asInstanceOf[S] + def decodeValue(row: UnsafeRow): V = { + val bytes = row.getBinary(0) + reuseRow.pointTo(bytes, bytes.length) + val value = rowToObjDeserializer.apply(reuseRow) + value } } object StateTypesEncoder { - def apply[GK]( + def apply[GK, V]( keySerializer: Serializer[GK], - stateName: String): StateTypesEncoder[GK] = { - new StateTypesEncoder[GK](keySerializer, stateName) + valEncoder: Encoder[V], + stateName: String): StateTypesEncoder[GK, V] = { + new StateTypesEncoder[GK, V](keySerializer, valEncoder, stateName) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala index 56a325a31e339..d5dd9fcaf4013 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala @@ -20,6 +20,7 @@ import java.util.UUID import org.apache.spark.TaskContext import org.apache.spark.internal.Logging +import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.streaming.{ListState, QueryInfo, StatefulProcessorHandle, ValueState} @@ -112,10 +113,10 @@ class StatefulProcessorHandleImpl( def getHandleState: StatefulProcessorHandleState = currState - override def getValueState[T](stateName: String): ValueState[T] = { + override def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] = { verify(currState == CREATED, s"Cannot create state variable with name=$stateName after " + "initialization is complete") - val resultState = new ValueStateImpl[T](store, stateName, keyEncoder) + val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder) resultState } @@ -132,10 +133,10 @@ class StatefulProcessorHandleImpl( store.removeColFamilyIfExists(stateName) } - override def getListState[T](stateName: String): ListState[T] = { + override def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T] = { verify(currState == CREATED, s"Cannot create state variable with name=$stateName after " + "initialization is complete") - val resultState = new ListStateImpl[T](store, stateName, keyEncoder) + val resultState = new ListStateImpl[T](store, stateName, keyEncoder, valEncoder) resultState } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala index a94a49d88325b..5d2b9881c78de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging +import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA} @@ -29,16 +30,18 @@ import org.apache.spark.sql.streaming.ValueState * @param store - reference to the StateStore instance to be used for storing state * @param stateName - name of logical state partition * @param keyEnc - Spark SQL encoder for key + * @param valEncoder - Spark SQL encoder for value * @tparam S - data type of object that will be stored */ class ValueStateImpl[S]( store: StateStore, stateName: String, - keyExprEnc: ExpressionEncoder[Any]) extends ValueState[S] with Logging { + keyExprEnc: ExpressionEncoder[Any], + valEncoder: Encoder[S]) extends ValueState[S] with Logging { private val keySerializer = keyExprEnc.createSerializer() - private val stateTypesEncoder = StateTypesEncoder(keySerializer, stateName) + private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder, stateName) store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, numColsPrefixKey = 0, VALUE_ROW_SCHEMA) @@ -57,7 +60,7 @@ class ValueStateImpl[S]( override def get(): S = { val retRow = getImpl() if (retRow != null) { - stateTypesEncoder.decodeValue[S](retRow) + stateTypesEncoder.decodeValue(retRow) } else { null.asInstanceOf[S] } diff --git a/sql/core/src/test/java/org/apache/spark/sql/execution/streaming/state/POJOTestClass.java b/sql/core/src/test/java/org/apache/spark/sql/execution/streaming/state/POJOTestClass.java new file mode 100644 index 0000000000000..0ba75f7789d0b --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/execution/streaming/state/POJOTestClass.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming.state; + +/** + * A POJO class used for tests of arbitrary state SQL encoder. + */ +public class POJOTestClass { + // Fields + private String name; + private int id; + + // Constructors + public POJOTestClass() { + // Default constructor + } + + public POJOTestClass(String name, int id) { + this.name = name; + this.id = id; + } + + // Getter methods + public String getName() { + return name; + } + + public int getId() { + return id; + } + + // Setter methods + public void setName(String name) { + this.name = name; + } + + public void setId(int id) { + this.id = id; + } + + // Additional methods if needed + public void incrementId() { + id++; + System.out.println(name + " is now " + id + "!"); + } + + // Override toString for better representation + @Override + public String toString() { + return "POJOTestClass{" + + "name='" + name + '\'' + + ", age=" + id + + '}'; + } + + // Override equals and hashCode for custom equality + @Override + public boolean equals(Object obj) { + POJOTestClass testObj = (POJOTestClass) obj; + return id == testObj.id && name.equals(testObj.name); + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala index be77f7a887c71..71462cb4b6435 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala @@ -33,6 +33,9 @@ import org.apache.spark.sql.streaming.ValueState import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +/** A case class for SQL encoder test purpose */ +case class TestClass(var id: Long, var name: String) + /** * Class that adds tests for single value ValueState types used in arbitrary stateful * operators such as transformWithState @@ -93,7 +96,7 @@ class ValueStateSuite extends SharedSparkSession Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) val stateName = "testState" - val testState: ValueState[Long] = handle.getValueState[Long]("testState") + val testState: ValueState[Long] = handle.getValueState[Long]("testState", Encoders.scalaLong) assert(ImplicitGroupingKeyTracker.getImplicitKeyOption.isEmpty) val ex = intercept[Exception] { testState.update(123) @@ -136,7 +139,7 @@ class ValueStateSuite extends SharedSparkSession val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) - val testState: ValueState[Long] = handle.getValueState[Long]("testState") + val testState: ValueState[Long] = handle.getValueState[Long]("testState", Encoders.scalaLong) ImplicitGroupingKeyTracker.setImplicitKey("test_key") testState.update(123) assert(testState.get() === 123) @@ -162,8 +165,10 @@ class ValueStateSuite extends SharedSparkSession val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) - val testState1: ValueState[Long] = handle.getValueState[Long]("testState1") - val testState2: ValueState[Long] = handle.getValueState[Long]("testState2") + val testState1: ValueState[Long] = handle.getValueState[Long]( + "testState1", Encoders.scalaLong) + val testState2: ValueState[Long] = handle.getValueState[Long]( + "testState2", Encoders.scalaLong) ImplicitGroupingKeyTracker.setImplicitKey("test_key") testState1.update(123) assert(testState1.get() === 123) @@ -217,4 +222,108 @@ class ValueStateSuite extends SharedSparkSession matchPVals = true ) } + + test("test SQL encoder - Value state operations for Primitive(Double) instances") { + tryWithProviderResource(newStoreProviderWithValueState(true)) { provider => + val store = provider.getStore(0) + val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), + Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) + + val testState: ValueState[Double] = handle.getValueState[Double]("testState", + Encoders.scalaDouble) + ImplicitGroupingKeyTracker.setImplicitKey("test_key") + testState.update(1.0) + assert(testState.get().equals(1.0)) + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + + testState.update(2.0) + assert(testState.get().equals(2.0)) + testState.update(3.0) + assert(testState.get().equals(3.0)) + + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + } + } + + test("test SQL encoder - Value state operations for Primitive(Long) instances") { + tryWithProviderResource(newStoreProviderWithValueState(true)) { provider => + val store = provider.getStore(0) + val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), + Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) + + val testState: ValueState[Long] = handle.getValueState[Long]("testState", + Encoders.scalaLong) + ImplicitGroupingKeyTracker.setImplicitKey("test_key") + testState.update(1L) + assert(testState.get().equals(1L)) + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + + testState.update(2L) + assert(testState.get().equals(2L)) + testState.update(3L) + assert(testState.get().equals(3L)) + + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + } + } + + test("test SQL encoder - Value state operations for case class instances") { + tryWithProviderResource(newStoreProviderWithValueState(true)) { provider => + val store = provider.getStore(0) + val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), + Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) + + val testState: ValueState[TestClass] = handle.getValueState[TestClass]("testState", + Encoders.product[TestClass]) + ImplicitGroupingKeyTracker.setImplicitKey("test_key") + testState.update(TestClass(1, "testcase1")) + assert(testState.get().equals(TestClass(1, "testcase1"))) + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + + testState.update(TestClass(2, "testcase2")) + assert(testState.get() === TestClass(2, "testcase2")) + testState.update(TestClass(3, "testcase3")) + assert(testState.get() === TestClass(3, "testcase3")) + + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + } + } + + test("test SQL encoder - Value state operations for POJO instances") { + tryWithProviderResource(newStoreProviderWithValueState(true)) { provider => + val store = provider.getStore(0) + val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(), + Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]]) + + val testState: ValueState[POJOTestClass] = handle.getValueState[POJOTestClass]("testState", + Encoders.bean(classOf[POJOTestClass])) + ImplicitGroupingKeyTracker.setImplicitKey("test_key") + testState.update(new POJOTestClass("testcase1", 1)) + assert(testState.get().equals(new POJOTestClass("testcase1", 1))) + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + + testState.update(new POJOTestClass("testcase2", 2)) + assert(testState.get().equals(new POJOTestClass("testcase2", 2))) + testState.update(new POJOTestClass("testcase3", 3)) + assert(testState.get().equals(new POJOTestClass("testcase3", 3))) + + testState.clear() + assert(!testState.exists()) + assert(testState.get() === null) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala index 3d085da4ab586..9572f7006f379 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkIllegalArgumentException +import org.apache.spark.sql.Encoders import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider} import org.apache.spark.sql.internal.SQLConf @@ -30,7 +31,7 @@ class TestListStateProcessor @transient var _listState: ListState[String] = _ override def init(outputMode: OutputMode): Unit = { - _listState = getHandle.getListState("testListState") + _listState = getHandle.getListState("testListState", Encoders.STRING) } override def handleInputRows( @@ -86,8 +87,8 @@ class ToggleSaveAndEmitProcessor @transient var _valueState: ValueState[Boolean] = _ override def init(outputMode: OutputMode): Unit = { - _listState = getHandle.getListState("testListState") - _valueState = getHandle.getValueState("testValueState") + _listState = getHandle.getListState("testListState", Encoders.STRING) + _valueState = getHandle.getValueState("testValueState", Encoders.scalaBoolean) } override def handleInputRows( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 8a87472a023a3..cc8c64c94c02f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.{SparkException, SparkRuntimeException} import org.apache.spark.internal.Logging +import org.apache.spark.sql.Encoders import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider, StateStoreMultipleColumnFamiliesNotSupportedException} import org.apache.spark.sql.internal.SQLConf @@ -32,7 +33,7 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S @transient private var _countState: ValueState[Long] = _ override def init(outputMode: OutputMode): Unit = { - _countState = getHandle.getValueState[Long]("countState") + _countState = getHandle.getValueState[Long]("countState", Encoders.scalaLong) } override def handleInputRows( @@ -59,8 +60,8 @@ class RunningCountMostRecentStatefulProcessor @transient private var _mostRecent: ValueState[String] = _ override def init(outputMode: OutputMode): Unit = { - _countState = getHandle.getValueState[Long]("countState") - _mostRecent = getHandle.getValueState[String]("mostRecent") + _countState = getHandle.getValueState[Long]("countState", Encoders.scalaLong) + _mostRecent = getHandle.getValueState[String]("mostRecent", Encoders.STRING) } override def handleInputRows( key: String, @@ -88,7 +89,7 @@ class MostRecentStatefulProcessorWithDeletion override def init(outputMode: OutputMode): Unit = { getHandle.deleteIfExists("countState") - _mostRecent = getHandle.getValueState[String]("mostRecent") + _mostRecent = getHandle.getValueState[String]("mostRecent", Encoders.STRING) } override def handleInputRows( @@ -116,7 +117,7 @@ class RunningCountStatefulProcessorWithError extends RunningCountStatefulProcess inputRows: Iterator[String], timerValues: TimerValues): Iterator[(String, String)] = { // Trying to create value state here should fail - _tempState = getHandle.getValueState[Long]("tempState") + _tempState = getHandle.getValueState[Long]("tempState", Encoders.scalaLong) Iterator.empty } } From 72a95bcad7f1906c97fb0971ed6338374ec3009d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 11 Mar 2024 09:34:12 +0900 Subject: [PATCH 22/50] [SPARK-47324][SQL] Add missing timestamp conversion for JDBC nested types ### What changes were proposed in this pull request? [SPARK-44280](https://issues.apache.org/jira/browse/SPARK-44280) added a new API convertJavaTimestampToTimestamp which is called only for plain timestamps. This PR makes it work for timestamps in arrays ### Why are the changes needed? data consistency/correctness ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #45435 from yaooqinn/SPARK-47324. Authored-by: Kent Yao Signed-off-by: Hyukjin Kwon --- .../sql/jdbc/PostgresIntegrationSuite.scala | 17 ++++--- .../datasources/jdbc/JdbcUtils.scala | 46 ++++++++----------- 2 files changed, 29 insertions(+), 34 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 2d1c0314f27b4..04e31679f3867 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -23,8 +23,7 @@ import java.text.SimpleDateFormat import java.time.{LocalDateTime, ZoneOffset} import java.util.Properties -import org.apache.spark.sql.Column -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, ShortType} import org.apache.spark.tags.DockerTest @@ -149,9 +148,12 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { |('2013-04-05 18:01:02.123456')""".stripMargin).executeUpdate() conn.prepareStatement("CREATE TABLE infinity_timestamp" + - "(id SERIAL PRIMARY KEY, timestamp_column TIMESTAMP);").executeUpdate() - conn.prepareStatement("INSERT INTO infinity_timestamp (timestamp_column)" + - " VALUES ('infinity'), ('-infinity');").executeUpdate() + "(id SERIAL PRIMARY KEY, timestamp_column TIMESTAMP, timestamp_array TIMESTAMP[])") + .executeUpdate() + conn.prepareStatement("INSERT INTO infinity_timestamp (timestamp_column, timestamp_array)" + + " VALUES ('infinity', ARRAY[TIMESTAMP 'infinity']), " + + "('-infinity', ARRAY[TIMESTAMP '-infinity'])") + .executeUpdate() conn.prepareStatement("CREATE DOMAIN not_null_text AS TEXT DEFAULT ''").executeUpdate() conn.prepareStatement("create table custom_type(type_array not_null_text[]," + @@ -447,10 +449,13 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(row.length == 2) val infinity = row(0).getAs[Timestamp]("timestamp_column") val negativeInfinity = row(1).getAs[Timestamp]("timestamp_column") + val infinitySeq = row(0).getAs[scala.collection.Seq[Timestamp]]("timestamp_array") + val negativeInfinitySeq = row(1).getAs[scala.collection.Seq[Timestamp]]("timestamp_array") val minTimeStamp = LocalDateTime.of(1, 1, 1, 0, 0, 0).toEpochSecond(ZoneOffset.UTC) val maxTimestamp = LocalDateTime.of(9999, 12, 31, 23, 59, 59).toEpochSecond(ZoneOffset.UTC) - assert(infinity.getTime == maxTimestamp) assert(negativeInfinity.getTime == minTimeStamp) + assert(infinitySeq.head.getTime == maxTimestamp) + assert(negativeInfinitySeq.head.getTime == minTimeStamp) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index b5e78ba32cd5c..a7bbb832a8390 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Connection, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException} +import java.math.{BigDecimal => JBigDecimal} +import java.sql.{Connection, Date, JDBCType, PreparedStatement, ResultSet, ResultSetMetaData, SQLException, Timestamp} import java.time.{Instant, LocalDate} import java.util import java.util.concurrent.TimeUnit @@ -414,7 +415,7 @@ object JdbcUtils extends Logging with SQLConfHelper { case DecimalType.Fixed(p, s) => (rs: ResultSet, row: InternalRow, pos: Int) => val decimal = - nullSafeConvert[java.math.BigDecimal](rs.getBigDecimal(pos + 1), d => Decimal(d, p, s)) + nullSafeConvert[JBigDecimal](rs.getBigDecimal(pos + 1), d => Decimal(d, p, s)) row.update(pos, decimal) case DoubleType => @@ -508,37 +509,22 @@ object JdbcUtils extends Logging with SQLConfHelper { case ArrayType(et, _) => val elementConversion = et match { - case TimestampType => - (array: Object) => - array.asInstanceOf[Array[java.sql.Timestamp]].map { timestamp => - nullSafeConvert(timestamp, fromJavaTimestamp) - } + case TimestampType => arrayConverter[Timestamp] { + (t: Timestamp) => fromJavaTimestamp(dialect.convertJavaTimestampToTimestamp(t)) + } case TimestampNTZType => - (array: Object) => - array.asInstanceOf[Array[java.sql.Timestamp]].map { timestamp => - nullSafeConvert(timestamp, (t: java.sql.Timestamp) => - localDateTimeToMicros(dialect.convertJavaTimestampToTimestampNTZ(t))) - } + arrayConverter[Timestamp] { + (t: Timestamp) => localDateTimeToMicros(dialect.convertJavaTimestampToTimestampNTZ(t)) + } case StringType => - (array: Object) => - // some underling types are not String such as uuid, inet, cidr, etc. - array.asInstanceOf[Array[java.lang.Object]] - .map(obj => if (obj == null) null else UTF8String.fromString(obj.toString)) - - case DateType => - (array: Object) => - array.asInstanceOf[Array[java.sql.Date]].map { date => - nullSafeConvert(date, fromJavaDate) - } + arrayConverter[Object]((obj: Object) => UTF8String.fromString(obj.toString)) + + case DateType => arrayConverter[Date](fromJavaDate) case dt: DecimalType => - (array: Object) => - array.asInstanceOf[Array[java.math.BigDecimal]].map { decimal => - nullSafeConvert[java.math.BigDecimal]( - decimal, d => Decimal(d, dt.precision, dt.scale)) - } + arrayConverter[java.math.BigDecimal](d => Decimal(d, dt.precision, dt.scale)) case LongType if metadata.contains("binarylong") => throw QueryExecutionErrors.unsupportedArrayElementTypeBasedOnBinaryError(dt) @@ -552,7 +538,7 @@ object JdbcUtils extends Logging with SQLConfHelper { (rs: ResultSet, row: InternalRow, pos: Int) => val array = nullSafeConvert[java.sql.Array]( input = rs.getArray(pos + 1), - array => new GenericArrayData(elementConversion.apply(array.getArray))) + array => new GenericArrayData(elementConversion(array.getArray))) row.update(pos, array) case _ => throw QueryExecutionErrors.unsupportedJdbcTypeError(dt.catalogString) @@ -566,6 +552,10 @@ object JdbcUtils extends Logging with SQLConfHelper { } } + private def arrayConverter[T](elementConvert: T => Any): Any => Any = (array: Any) => { + array.asInstanceOf[Array[T]].map(e => nullSafeConvert(e, elementConvert)) + } + // A `JDBCValueSetter` is responsible for setting a value from `Row` into a field for // `PreparedStatement`. The last argument `Int` means the index for the value to be set // in the SQL statement and also used for the value in `Row`. From 66d6a23dc3b71dbeb2f474e9c375ddd2dc55e382 Mon Sep 17 00:00:00 2001 From: Richard Chen Date: Mon, 11 Mar 2024 13:32:36 +0500 Subject: [PATCH 23/50] [SPARK-45827][SQL] Add variant singleton type for Java ### What changes were proposed in this pull request? Adds the Variant singleton ### Why are the changes needed? Dependent projects (i.e. Delta kernel) require access to this singleton ### 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 #45455 from richardc-db/add_variant_singleton_type. Authored-by: Richard Chen Signed-off-by: Max Gekk --- .../src/main/java/org/apache/spark/sql/types/DataTypes.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/api/src/main/java/org/apache/spark/sql/types/DataTypes.java b/sql/api/src/main/java/org/apache/spark/sql/types/DataTypes.java index 9454c3c259856..32c20dedac4c6 100644 --- a/sql/api/src/main/java/org/apache/spark/sql/types/DataTypes.java +++ b/sql/api/src/main/java/org/apache/spark/sql/types/DataTypes.java @@ -99,6 +99,11 @@ public class DataTypes { */ public static final DataType NullType = NullType$.MODULE$; + /** + * Gets the VariantType object. + */ + public static final DataType VariantType = VariantType$.MODULE$; + /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). * The field of {@code containsNull} is set to {@code true}. From 2a51242b1bdb4a1d469fc81dc0d21c4cde305753 Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Mon, 11 Mar 2024 18:14:07 +0800 Subject: [PATCH 24/50] [MINOR][DOCS][SQL] Fix doc comment for coalescePartitions.parallelismFirst on sql-performance-tuning page We missed that this doc comment is duplicated while fixing it in https://github.com/apache/spark/pull/45437 ### What changes were proposed in this pull request? Documentation fix. ### Why are the changes needed? Current doc is wrong. ### 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 #45458 from eejbyfeldt/fix-minor-doc-comment-on-parallism-first. Authored-by: Emil Ejbyfeldt Signed-off-by: Kent Yao --- docs/sql-performance-tuning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md index 3f523b48e252c..b443e3d9c5f59 100644 --- a/docs/sql-performance-tuning.md +++ b/docs/sql-performance-tuning.md @@ -278,7 +278,7 @@ This feature coalesces the post shuffle partitions based on the map output stati spark.sql.adaptive.coalescePartitions.parallelismFirst true - When true, Spark ignores the target size specified by spark.sql.adaptive.advisoryPartitionSizeInBytes (default 64MB) when coalescing contiguous shuffle partitions, and only respect the minimum partition size specified by spark.sql.adaptive.coalescePartitions.minPartitionSize (default 1MB), to maximize the parallelism. This is to avoid performance regressions when enabling adaptive query execution. It's recommended to set this config to true on a busy cluster to make resource utilization more efficient (not many small tasks). + When true, Spark ignores the target size specified by spark.sql.adaptive.advisoryPartitionSizeInBytes (default 64MB) when coalescing contiguous shuffle partitions, and only respect the minimum partition size specified by spark.sql.adaptive.coalescePartitions.minPartitionSize (default 1MB), to maximize the parallelism. This is to avoid performance regressions when enabling adaptive query execution. It's recommended to set this config to false on a busy cluster to make resource utilization more efficient (not many small tasks). 3.2.0 From f350b760ea117a2d670720e37ba7be3fd8c0de0a Mon Sep 17 00:00:00 2001 From: Zhen Wang <643348094@qq.com> Date: Mon, 11 Mar 2024 19:33:43 +0800 Subject: [PATCH 25/50] [SPARK-47270][SQL] Dataset.isEmpty projects CommandResults locally ### What changes were proposed in this pull request? Similar to #40779, `Dataset.isEmpty` should also not trigger job execution on CommandResults. This PR converts `CommandResult` to `LocalRelation` in `Dataset.isEmpty` method. ### Why are the changes needed? A simple `spark.sql("show tables").isEmpty` shouldn not require an executor. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added new UT. ### Was this patch authored or co-authored using generative AI tooling? No Closes #45373 from wForget/SPARK-47270. Authored-by: Zhen Wang <643348094@qq.com> Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/Dataset.scala | 22 ++++++++++++------- .../org/apache/spark/sql/DatasetSuite.scala | 19 ++++++++++++++++ 2 files changed, 33 insertions(+), 8 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 f0c9f7ae53fc4..c2e51b574df77 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 @@ -271,13 +271,7 @@ class Dataset[T] private[sql]( private[sql] def getRows( numRows: Int, truncate: Int): Seq[Seq[String]] = { - val newDf = logicalPlan match { - case c: CommandResult => - // Convert to `LocalRelation` and let `ConvertToLocalRelation` do the casting locally to - // avoid triggering a job - Dataset.ofRows(sparkSession, LocalRelation(c.output, c.rows)) - case _ => toDF() - } + val newDf = commandResultOptimized.toDF() val castCols = newDf.logicalPlan.output.map { col => Column(ToPrettyString(col)) } @@ -655,7 +649,8 @@ class Dataset[T] private[sql]( * @group basic * @since 2.4.0 */ - def isEmpty: Boolean = withAction("isEmpty", select().limit(1).queryExecution) { plan => + def isEmpty: Boolean = withAction("isEmpty", + commandResultOptimized.select().limit(1).queryExecution) { plan => plan.executeTake(1).isEmpty } @@ -4471,6 +4466,17 @@ class Dataset[T] private[sql]( } } + /** Returns a optimized plan for CommandResult, convert to `LocalRelation`. */ + private def commandResultOptimized: Dataset[T] = { + logicalPlan match { + case c: CommandResult => + // Convert to `LocalRelation` and let `ConvertToLocalRelation` do the casting locally to + // avoid triggering a job + Dataset(sparkSession, LocalRelation(c.output, c.rows)) + case _ => this + } + } + /** Convert to an RDD of serialized ArrowRecordBatches. */ private[sql] def toArrowBatchRdd(plan: SparkPlan): RDD[Array[Byte]] = { val schemaCaptured = this.schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index bb18769a23b47..91057dcc98e09 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -2719,6 +2719,25 @@ class DatasetSuite extends QueryTest checkDataset(ds.map(t => t), WithSet(0, HashSet("foo", "bar")), WithSet(1, HashSet("bar", "zoo"))) } + + test("SPARK-47270: isEmpty does not trigger job execution on CommandResults") { + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { + withTable("t1") { + sql("create table t1(c int) using parquet") + + @volatile var jobCounter = 0 + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + jobCounter += 1 + } + } + withListener(spark.sparkContext, listener) { _ => + sql("show tables").isEmpty + } + assert(jobCounter === 0) + } + } + } } class DatasetLargeResultCollectingSuite extends QueryTest From d5f42fc6f1863a11d9f22f4179a7edbaf41210dd Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 11 Mar 2024 20:08:50 +0800 Subject: [PATCH 26/50] [SPARK-47337][SQL][DOCKER] Upgrade DB2 docker image version to 11.5.8.0 ### What changes were proposed in this pull request? This PR upgrades the DB2 Docker image version to 11.5.8.0 used by docker-integration tests https://hub.docker.com/r/ibmcom/db2/tags ### Why are the changes needed? refreshing test dependencies ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? `DB2..Suite`s ### Was this patch authored or co-authored using generative AI tooling? no Closes #45456 from yaooqinn/SPARK-47337. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../spark/sql/jdbc/DB2DatabaseOnDocker.scala | 34 +++++++++++++++++++ .../spark/sql/jdbc/DB2IntegrationSuite.scala | 20 ++--------- .../sql/jdbc/DB2KrbIntegrationSuite.scala | 19 +++-------- .../sql/jdbc/v2/DB2IntegrationSuite.scala | 23 +++---------- .../spark/sql/jdbc/v2/DB2NamespaceSuite.scala | 23 +++---------- 5 files changed, 49 insertions(+), 70 deletions(-) create mode 100644 connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2DatabaseOnDocker.scala diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2DatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2DatabaseOnDocker.scala new file mode 100644 index 0000000000000..5cfd998b20a75 --- /dev/null +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2DatabaseOnDocker.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +class DB2DatabaseOnDocker extends DatabaseOnDocker { + override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.8.0") + override val env = Map( + "DB2INST1_PASSWORD" -> "rootpass", + "LICENSE" -> "accept", + "DBNAME" -> "foo", + "ARCHIVE_LOGS" -> "false", + "AUTOCONFIG" -> "false" + ) + override val usesIpc = false + override val jdbcPort: Int = 50000 + override val privileged = true + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore +} diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala index df29326848802..cedb33d491fbc 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala @@ -29,30 +29,16 @@ import org.apache.spark.sql.types.{BooleanType, ByteType, ShortType, StructType} import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.6.0a): + * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.8.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.6.0a + * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.8.0 * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.DB2IntegrationSuite" * }}} */ @DockerTest class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.6.0a") - override val env = Map( - "DB2INST1_PASSWORD" -> "rootpass", - "LICENSE" -> "accept", - "DBNAME" -> "foo", - "ARCHIVE_LOGS" -> "false", - "AUTOCONFIG" -> "false" - ) - override val usesIpc = false - override val jdbcPort: Int = 50000 - override val privileged = true - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore - } + override val db = new DB2DatabaseOnDocker override val connectionTimeout = timeout(3.minutes) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala index d221fbfab6229..abb683c064955 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala @@ -31,9 +31,9 @@ import org.apache.spark.sql.execution.datasources.jdbc.connection.{DB2Connection import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.6.0a): + * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.8.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.6.0a + * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.8.0 * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly *DB2KrbIntegrationSuite" * }}} @@ -43,19 +43,8 @@ class DB2KrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val userName = s"db2/$dockerIp" override protected val keytabFileName = "db2.keytab" - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.6.0a") - override val env = Map( - "DB2INST1_PASSWORD" -> "rootpass", - "LICENSE" -> "accept", - "DBNAME" -> "db2", - "ARCHIVE_LOGS" -> "false", - "AUTOCONFIG" -> "false" - ) - override val usesIpc = false - override val jdbcPort = 50000 - override val privileged = true - override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:db2://$ip:$port/db2" + override val db = new DB2DatabaseOnDocker { + override def getJdbcUrl(ip: String, port: Int): String = s"jdbc:db2://$ip:$port/foo" override def getJdbcProperties() = { val options = new JDBCOptions(Map[String, String]( JDBCOptions.JDBC_URL -> getJdbcUrl(dockerIp, externalPort), diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala index 661b1277e9f03..c3ec7e1925fa3 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2IntegrationSuite.scala @@ -25,14 +25,14 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkConf import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog -import org.apache.spark.sql.jdbc.DatabaseOnDocker +import org.apache.spark.sql.jdbc.DB2DatabaseOnDocker import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.6.0a): + * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.8.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.6.0a + * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.8.0 * ./build/sbt -Pdocker-integration-tests "testOnly *v2.DB2IntegrationSuite" * }}} */ @@ -51,22 +51,7 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { override val catalogName: String = "db2" override val namespaceOpt: Option[String] = Some("DB2INST1") - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.6.0a") - override val env = Map( - "DB2INST1_PASSWORD" -> "rootpass", - "LICENSE" -> "accept", - "DBNAME" -> "foo", - "ARCHIVE_LOGS" -> "false", - "AUTOCONFIG" -> "false" - ) - override val usesIpc = false - override val jdbcPort: Int = 50000 - override val privileged = true - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore - } - + override val db = new DB2DatabaseOnDocker override val connectionTimeout = timeout(3.minutes) override def sparkConf: SparkConf = super.sparkConf diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2NamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2NamespaceSuite.scala index 3bf3e785cdae8..528b26e61e761 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2NamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/DB2NamespaceSuite.scala @@ -21,35 +21,20 @@ import java.sql.Connection import scala.jdk.CollectionConverters._ -import org.apache.spark.sql.jdbc.{DatabaseOnDocker, DockerJDBCIntegrationSuite} +import org.apache.spark.sql.jdbc.{DB2DatabaseOnDocker, DockerJDBCIntegrationSuite} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.6.0a): + * To run this test suite for a specific version (e.g., ibmcom/db2:11.5.8.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.6.0a + * ENABLE_DOCKER_INTEGRATION_TESTS=1 DB2_DOCKER_IMAGE_NAME=ibmcom/db2:11.5.8.0 * ./build/sbt -Pdocker-integration-tests "testOnly *v2.DB2NamespaceSuite" * }}} */ @DockerTest class DB2NamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespaceTest { - override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("DB2_DOCKER_IMAGE_NAME", "ibmcom/db2:11.5.6.0a") - override val env = Map( - "DB2INST1_PASSWORD" -> "rootpass", - "LICENSE" -> "accept", - "DBNAME" -> "db2foo", - "ARCHIVE_LOGS" -> "false", - "AUTOCONFIG" -> "false" - ) - override val usesIpc = false - override val jdbcPort: Int = 50000 - override val privileged = true - override def getJdbcUrl(ip: String, port: Int): String = - s"jdbc:db2://$ip:$port/db2foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore - } - + override val db = new DB2DatabaseOnDocker val map = new CaseInsensitiveStringMap( Map("url" -> db.getJdbcUrl(dockerIp, externalPort), "driver" -> "com.ibm.db2.jcc.DB2Driver").asJava) From 427f74066d57cfeba2429b01bbe7ee7d5bb7ad12 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Mon, 11 Mar 2024 17:41:18 +0500 Subject: [PATCH 27/50] [SPARK-47328][SQL] Rename UCS_BASIC collation to UTF8_BINARY ### What changes were proposed in this pull request? Change utf8 collation names from 'UCS_BASIC' to 'UTF8_BINARY'. ### Why are the changes needed? To have the name better describe the actual meaning of the default collation. ### Does this PR introduce _any_ user-facing change? Yes, now we have different names for utf8 collation ### How was this patch tested? With existing unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45442 from stefankandic/renameUCS. Authored-by: Stefan Kandic Signed-off-by: Max Gekk --- .../sql/catalyst/util/CollationFactory.java | 4 +- .../apache/spark/unsafe/types/UTF8String.java | 2 +- .../unsafe/types/CollationFactorySuite.scala | 54 +++--- .../queries/select_collated_string.json | 2 +- .../queries/select_collated_string.proto.bin | Bin 63 -> 65 bytes .../planner/SparkConnectProtoSuite.scala | 2 +- python/pyspark/sql/tests/test_types.py | 10 +- python/pyspark/sql/types.py | 2 +- .../apache/spark/sql/types/StringType.scala | 4 +- .../expressions/collationExpressions.scala | 6 +- .../CollationExpressionSuite.scala | 31 +-- .../MergeScalarSubqueriesSuite.scala | 8 +- .../sql-functions/sql-expression-schema.md | 2 +- .../analyzer-results/collations.sql.out | 110 +++++------ .../resources/sql-tests/inputs/collations.sql | 46 ++--- .../sql-tests/results/collations.sql.out | 42 ++-- .../org/apache/spark/sql/CollationSuite.scala | 180 +++++++++--------- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- 18 files changed, 255 insertions(+), 252 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java index ed8bfb886c6e7..151a1c9ddbbec 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java @@ -117,7 +117,7 @@ public Collation( // No custom comparators will be used for this collation. // Instead, we rely on byte for byte comparison. collationTable[0] = new Collation( - "UCS_BASIC", + "UTF8_BINARY", null, UTF8String::binaryCompare, "1.0", @@ -127,7 +127,7 @@ public Collation( // Case-insensitive UTF8 binary collation. // TODO: Do in place comparisons instead of creating new strings. collationTable[1] = new Collation( - "UCS_BASIC_LCASE", + "UTF8_BINARY_LCASE", null, (s1, s2) -> s1.toLowerCase().binaryCompare(s2.toLowerCase()), "1.0", diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index ea523760ad8e0..d5ca8ee629801 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1456,7 +1456,7 @@ public int compareTo(@Nonnull final UTF8String other) { } /** - * Binary comparison of two UTF8String. Can only be used for default UCS_BASIC collation. + * Binary comparison of two UTF8String. Can only be used for default UTF8_BINARY collation. */ public int binaryCompare(final UTF8String other) { return ByteArray.compareBinary( diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala index 5d760ba795df0..f9927b94fd42c 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala @@ -29,13 +29,13 @@ import org.apache.spark.unsafe.types.UTF8String.{fromString => toUTF8} class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ignore funsuite test("collationId stability") { - val ucsBasic = fetchCollation(0) - assert(ucsBasic.collationName == "UCS_BASIC") - assert(ucsBasic.isBinaryCollation) + val utf8Binary = fetchCollation(0) + assert(utf8Binary.collationName == "UTF8_BINARY") + assert(utf8Binary.isBinaryCollation) - val ucsBasicLcase = fetchCollation(1) - assert(ucsBasicLcase.collationName == "UCS_BASIC_LCASE") - assert(!ucsBasicLcase.isBinaryCollation) + val utf8BinaryLcase = fetchCollation(1) + assert(utf8BinaryLcase.collationName == "UTF8_BINARY_LCASE") + assert(!utf8BinaryLcase.isBinaryCollation) val unicode = fetchCollation(2) assert(unicode.collationName == "UNICODE") @@ -48,27 +48,27 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig test("fetch invalid collation name") { val error = intercept[SparkException] { - fetchCollation("UCS_BASIS") + fetchCollation("UTF8_BS") } assert(error.getErrorClass === "COLLATION_INVALID_NAME") assert(error.getMessageParameters.asScala === - Map("proposal" -> "UCS_BASIC", "collationName" -> "UCS_BASIS")) + Map("proposal" -> "UTF8_BINARY", "collationName" -> "UTF8_BS")) } case class CollationTestCase[R](collationName: String, s1: String, s2: String, expectedResult: R) test("collation aware equality and hash") { val checks = Seq( - CollationTestCase("UCS_BASIC", "aaa", "aaa", true), - CollationTestCase("UCS_BASIC", "aaa", "AAA", false), - CollationTestCase("UCS_BASIC", "aaa", "bbb", false), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "aaa", true), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "AAA", true), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "AaA", true), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "AaA", true), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "aa", false), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "bbb", false), + CollationTestCase("UTF8_BINARY", "aaa", "aaa", true), + CollationTestCase("UTF8_BINARY", "aaa", "AAA", false), + CollationTestCase("UTF8_BINARY", "aaa", "bbb", false), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "aaa", true), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AAA", true), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", true), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", true), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "aa", false), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "bbb", false), CollationTestCase("UNICODE", "aaa", "aaa", true), CollationTestCase("UNICODE", "aaa", "AAA", false), CollationTestCase("UNICODE", "aaa", "bbb", false), @@ -89,16 +89,16 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig test("collation aware compare") { val checks = Seq( - CollationTestCase("UCS_BASIC", "aaa", "aaa", 0), - CollationTestCase("UCS_BASIC", "aaa", "AAA", 1), - CollationTestCase("UCS_BASIC", "aaa", "bbb", -1), - CollationTestCase("UCS_BASIC", "aaa", "BBB", 1), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "aaa", 0), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "AAA", 0), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "AaA", 0), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "AaA", 0), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "aa", 1), - CollationTestCase("UCS_BASIC_LCASE", "aaa", "bbb", -1), + CollationTestCase("UTF8_BINARY", "aaa", "aaa", 0), + CollationTestCase("UTF8_BINARY", "aaa", "AAA", 1), + CollationTestCase("UTF8_BINARY", "aaa", "bbb", -1), + CollationTestCase("UTF8_BINARY", "aaa", "BBB", 1), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "aaa", 0), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AAA", 0), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", 0), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", 0), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "aa", 1), + CollationTestCase("UTF8_BINARY_LCASE", "aaa", "bbb", -1), CollationTestCase("UNICODE", "aaa", "aaa", 0), CollationTestCase("UNICODE", "aaa", "AAA", -1), CollationTestCase("UNICODE", "aaa", "bbb", -1), diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json index 00644e072190e..7bf1e0a7bb10c 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json @@ -8,7 +8,7 @@ "planId": "0" }, "localRelation": { - "schema": "struct\u003cs:string COLLATE UCS_BASIC_LCASE\u003e" + "schema": "struct\u003cs:string COLLATE UTF8_BINARY_LCASE\u003e" } }, "expressions": [{ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin index b247463daf43d9162d0107fa10f9d5ab24d08f57..bbe6d999c47cdb88ff824a52212ecb43c4c9eb4c 100644 GIT binary patch delta 47 zcmcC_6yRbKV3e}vG6K>JQ943e69ptCLfkCkojm;0k~%f AoB#j- delta 45 ycmZ>?7vN$NV3e}tG62#HQCdP869pv2oP*UScRCm7>fa;RtNI{ diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index b989f5027cf92..632e2308fc76b 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -1036,7 +1036,7 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { } test("SPARK-47144: Collated string") { - Seq("UCS_BASIC", "UCS_BASIC_LCASE", "UNICODE", "UNICODE_CI").map(collationName => + Seq("UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI").map(collationName => Seq( s"select 'abc' collate $collationName", s"select collation('abc' collate $collationName)").map(query => diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index a0dfdce1a96e8..f4a857ac3ee0c 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -862,11 +862,11 @@ def test_parse_datatype_string(self): if k != "varchar" and k != "char": self.assertEqual(t(), _parse_datatype_string(k)) self.assertEqual(IntegerType(), _parse_datatype_string("int")) - self.assertEqual(StringType(), _parse_datatype_string("string COLLATE UCS_BASIC")) + self.assertEqual(StringType(), _parse_datatype_string("string COLLATE UTF8_BINARY")) self.assertEqual(StringType(0), _parse_datatype_string("string")) - self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UCS_BASIC")) - self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UCS_BASIC")) - self.assertEqual(StringType(1), _parse_datatype_string("string COLLATE UCS_BASIC_LCASE")) + self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UTF8_BINARY")) + self.assertEqual(StringType(0), _parse_datatype_string("string COLLATE UTF8_BINARY")) + self.assertEqual(StringType(1), _parse_datatype_string("string COLLATE UTF8_BINARY_LCASE")) self.assertEqual(StringType(2), _parse_datatype_string("string COLLATE UNICODE")) self.assertEqual(StringType(2), _parse_datatype_string("string COLLATE `UNICODE`")) self.assertEqual(StringType(3), _parse_datatype_string("string COLLATE UNICODE_CI")) @@ -1410,7 +1410,7 @@ def test_from_ddl(self): def test_collated_string(self): dfs = [ - self.spark.sql("SELECT 'abc' collate UCS_BASIC_LCASE"), + self.spark.sql("SELECT 'abc' collate UTF8_BINARY_LCASE"), self.spark.createDataFrame([], StructType([StructField("id", StringType(1))])), ] for df in dfs: diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index a30f41ae40239..0b98ad346576d 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -254,7 +254,7 @@ class StringType(AtomicType): the collation id number. """ - collationNames = ["UCS_BASIC", "UCS_BASIC_LCASE", "UNICODE", "UNICODE_CI"] + collationNames = ["UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI"] def __init__(self, collationId: int = 0): self.collationId = collationId diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index b434c98edd2a5..35dfbf758ad41 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.CollationFactory @Stable class StringType private(val collationId: Int) extends AtomicType with Serializable { /** - * Returns whether assigned collation is the default spark collation (UCS_BASIC). + * Returns whether assigned collation is the default spark collation (UTF8_BINARY). */ def isDefaultCollation: Boolean = collationId == CollationFactory.DEFAULT_COLLATION_ID @@ -43,7 +43,7 @@ class StringType private(val collationId: Int) extends AtomicType with Serializa /** * Type name that is shown to the customer. - * If this is an UCS_BASIC collation output is `string` due to backwards compatibility. + * If this is an UTF8_BINARY collation output is `string` due to backwards compatibility. */ override def typeName: String = if (isDefaultCollation) "string" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala index 99237d978deea..b0f77bad44831 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala @@ -37,8 +37,8 @@ import org.apache.spark.sql.types._ Examples: > SET spark.sql.collation.enabled=true; spark.sql.collation.enabled true - > SELECT COLLATION('Spark SQL' _FUNC_ UCS_BASIC_LCASE); - UCS_BASIC_LCASE + > SELECT COLLATION('Spark SQL' _FUNC_ UTF8_BINARY_LCASE); + UTF8_BINARY_LCASE > SET spark.sql.collation.enabled=false; spark.sql.collation.enabled false """, @@ -105,7 +105,7 @@ case class Collate(child: Expression, collationName: String) > SET spark.sql.collation.enabled=true; spark.sql.collation.enabled true > SELECT _FUNC_('Spark SQL'); - UCS_BASIC + UTF8_BINARY > SET spark.sql.collation.enabled=false; spark.sql.collation.enabled false """, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala index 35b0f7f5f3267..a9781e140afe6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala @@ -23,55 +23,58 @@ import org.apache.spark.sql.types._ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { test("validate default collation") { - val collationId = CollationFactory.collationNameToId("UCS_BASIC") + val collationId = CollationFactory.collationNameToId("UTF8_BINARY") assert(collationId == 0) - val collateExpr = Collate(Literal("abc"), "UCS_BASIC") + val collateExpr = Collate(Literal("abc"), "UTF8_BINARY") assert(collateExpr.dataType === StringType(collationId)) collateExpr.dataType.asInstanceOf[StringType].collationId == 0 checkEvaluation(collateExpr, "abc") } test("collate against literal") { - val collateExpr = Collate(Literal("abc"), "UCS_BASIC_LCASE") - val collationId = CollationFactory.collationNameToId("UCS_BASIC_LCASE") + val collateExpr = Collate(Literal("abc"), "UTF8_BINARY_LCASE") + val collationId = CollationFactory.collationNameToId("UTF8_BINARY_LCASE") assert(collateExpr.dataType == StringType(collationId)) checkEvaluation(collateExpr, "abc") } test("check input types") { - val collateExpr = Collate(Literal("abc"), "UCS_BASIC") + val collateExpr = Collate(Literal("abc"), "UTF8_BINARY") assert(collateExpr.checkInputDataTypes().isSuccess) val collateExprExplicitDefault = - Collate(Literal.create("abc", StringType(0)), "UCS_BASIC") + Collate(Literal.create("abc", StringType(0)), "UTF8_BINARY") assert(collateExprExplicitDefault.checkInputDataTypes().isSuccess) val collateExprExplicitNonDefault = - Collate(Literal.create("abc", StringType(1)), "UCS_BASIC") + Collate(Literal.create("abc", StringType(1)), "UTF8_BINARY") assert(collateExprExplicitNonDefault.checkInputDataTypes().isSuccess) - val collateOnNull = Collate(Literal.create(null, StringType(1)), "UCS_BASIC") + val collateOnNull = Collate(Literal.create(null, StringType(1)), "UTF8_BINARY") assert(collateOnNull.checkInputDataTypes().isSuccess) - val collateOnInt = Collate(Literal(1), "UCS_BASIC") + val collateOnInt = Collate(Literal(1), "UTF8_BINARY") assert(collateOnInt.checkInputDataTypes().isFailure) } test("collate on non existing collation") { checkError( - exception = intercept[SparkException] { Collate(Literal("abc"), "UCS_BASIS") }, + exception = intercept[SparkException] { Collate(Literal("abc"), "UTF8_BS") }, errorClass = "COLLATION_INVALID_NAME", sqlState = "42704", - parameters = Map("proposal" -> "UCS_BASIC", "collationName" -> "UCS_BASIS")) + parameters = Map("proposal" -> "UTF8_BINARY", "collationName" -> "UTF8_BS")) } test("collation on non-explicit default collation") { - checkEvaluation(Collation(Literal("abc")).replacement, "UCS_BASIC") + checkEvaluation(Collation(Literal("abc")).replacement, "UTF8_BINARY") } test("collation on explicitly collated string") { - checkEvaluation(Collation(Literal.create("abc", StringType(1))).replacement, "UCS_BASIC_LCASE") checkEvaluation( - Collation(Collate(Literal("abc"), "UCS_BASIC_LCASE")).replacement, "UCS_BASIC_LCASE") + Collation(Literal.create("abc", StringType(1))).replacement, + "UTF8_BINARY_LCASE") + checkEvaluation( + Collation(Collate(Literal("abc"), "UTF8_BINARY_LCASE")).replacement, + "UTF8_BINARY_LCASE") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala index a74ea21a56e0c..b640344658d40 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala @@ -37,8 +37,8 @@ class MergeScalarSubqueriesSuite extends PlanTest { val testRelation = LocalRelation(Symbol("a").int, Symbol("b").int, Symbol("c").string) val testRelationWithNonBinaryCollation = LocalRelation( - Symbol("ucs_basic").string("UCS_BASIC"), - Symbol("ucs_basic_lcase").string("UCS_BASIC_LCASE")) + Symbol("utf8_binary").string("UTF8_BINARY"), + Symbol("utf8_binary_lcase").string("UTF8_BINARY_LCASE")) private def definitionNode(plan: LogicalPlan, cteIndex: Int) = { CTERelationDef(plan, cteIndex, underSubquery = true) @@ -202,9 +202,9 @@ class MergeScalarSubqueriesSuite extends PlanTest { test("Avoid merge when mixing sort and hash aggs") { val subquery1 = ScalarSubquery(testRelationWithNonBinaryCollation.groupBy( - Symbol("ucs_basic"))(max(Symbol("ucs_basic")).as("max_ucs_basic"))) + Symbol("utf8_binary"))(max(Symbol("utf8_binary")).as("max_utf8_binary"))) val subquery2 = ScalarSubquery(testRelationWithNonBinaryCollation.groupBy( - Symbol("ucs_basic_lcase"))(max(Symbol("ucs_basic_lcase")).as("ucs_basic_lcase"))) + Symbol("utf8_binary_lcase"))(max(Symbol("utf8_binary_lcase")).as("utf8_binary_lcase"))) val originalQuery = testRelationWithNonBinaryCollation.select(subquery1, subquery2) Optimize.execute(originalQuery.analyze).collect { case WithCTE(_, _) => fail("Should not have merged") diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 33c26fe3c036b..bd1b6f0cb7537 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -81,7 +81,7 @@ | org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | struct | | org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | struct | | org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT coalesce(NULL, 1, NULL) | struct | -| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UCS_BASIC_LCASE) | struct | +| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UTF8_BINARY_LCASE) | struct | | org.apache.spark.sql.catalyst.expressions.Collation | collation | SELECT collation('Spark SQL') | struct | | org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT concat('Spark', 'SQL') | struct | | org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT concat_ws(' ', 'Spark', 'SQL') | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index 794185c5cba44..102755f68ad1d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -1,6 +1,6 @@ -- Automatically generated by SQLQueryTestSuite -- !query -create table t1(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet +create table t1(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false @@ -8,102 +8,102 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query insert into t1 values('aaa', 'aaa') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string COLLATE UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t1 values('AAA', 'AAA') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string COLLATE UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t1 values('bbb', 'bbb') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string COLLATE UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t1 values('BBB', 'BBB') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string COLLATE UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query -select count(*) from t1 group by ucs_basic +select count(*) from t1 group by utf8_binary -- !query analysis -Aggregate [ucs_basic#x], [count(1) AS count(1)#xL] +Aggregate [utf8_binary#x], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -select count(*) from t1 group by ucs_basic_lcase +select count(*) from t1 group by utf8_binary_lcase -- !query analysis -Aggregate [ucs_basic_lcase#x], [count(1) AS count(1)#xL] +Aggregate [utf8_binary_lcase#x], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -select * from t1 where ucs_basic = 'aaa' +select * from t1 where utf8_binary = 'aaa' -- !query analysis -Project [ucs_basic#x, ucs_basic_lcase#x] -+- Filter (ucs_basic#x = aaa) +Project [utf8_binary#x, utf8_binary_lcase#x] ++- Filter (utf8_binary#x = aaa) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -select * from t1 where ucs_basic_lcase = 'aaa' collate ucs_basic_lcase +select * from t1 where utf8_binary_lcase = 'aaa' collate utf8_binary_lcase -- !query analysis -Project [ucs_basic#x, ucs_basic_lcase#x] -+- Filter (ucs_basic_lcase#x = collate(aaa, ucs_basic_lcase)) +Project [utf8_binary#x, utf8_binary_lcase#x] ++- Filter (utf8_binary_lcase#x = collate(aaa, utf8_binary_lcase)) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -select * from t1 where ucs_basic < 'bbb' +select * from t1 where utf8_binary < 'bbb' -- !query analysis -Project [ucs_basic#x, ucs_basic_lcase#x] -+- Filter (ucs_basic#x < bbb) +Project [utf8_binary#x, utf8_binary_lcase#x] ++- Filter (utf8_binary#x < bbb) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -select * from t1 where ucs_basic_lcase < 'bbb' collate ucs_basic_lcase +select * from t1 where utf8_binary_lcase < 'bbb' collate utf8_binary_lcase -- !query analysis -Project [ucs_basic#x, ucs_basic_lcase#x] -+- Filter (ucs_basic_lcase#x < collate(bbb, ucs_basic_lcase)) +Project [utf8_binary#x, utf8_binary_lcase#x] ++- Filter (utf8_binary_lcase#x < collate(bbb, utf8_binary_lcase)) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -select l.ucs_basic, r.ucs_basic_lcase from t1 l join t1 r on l.ucs_basic_lcase = r.ucs_basic_lcase +select l.utf8_binary, r.utf8_binary_lcase from t1 l join t1 r on l.utf8_binary_lcase = r.utf8_binary_lcase -- !query analysis -Project [ucs_basic#x, ucs_basic_lcase#x] -+- Join Inner, (ucs_basic_lcase#x = ucs_basic_lcase#x) +Project [utf8_binary#x, utf8_binary_lcase#x] ++- Join Inner, (utf8_binary_lcase#x = utf8_binary_lcase#x) :- SubqueryAlias l : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet +- SubqueryAlias r +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query -create table t2(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet +create table t2(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false @@ -111,28 +111,28 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false -- !query insert into t2 values('aaa', 'aaa') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_binary_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string COLLATE UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t2 values('bbb', 'bbb') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [ucs_basic, ucs_basic_lcase] -+- Project [cast(col1#x as string) AS ucs_basic#x, cast(col2#x as string COLLATE UCS_BASIC_LCASE) AS ucs_basic_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_binary_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string COLLATE UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query -select * from t1 anti join t2 on t1.ucs_basic_lcase = t2.ucs_basic_lcase +select * from t1 anti join t2 on t1.utf8_binary_lcase = t2.utf8_binary_lcase -- !query analysis -Project [ucs_basic#x, ucs_basic_lcase#x] -+- Join LeftAnti, (ucs_basic_lcase#x = ucs_basic_lcase#x) +Project [utf8_binary#x, utf8_binary_lcase#x] ++- Join LeftAnti, (utf8_binary_lcase#x = utf8_binary_lcase#x) :- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[ucs_basic#x,ucs_basic_lcase#x] parquet + : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet +- SubqueryAlias spark_catalog.default.t2 - +- Relation spark_catalog.default.t2[ucs_basic#x,ucs_basic_lcase#x] parquet + +- Relation spark_catalog.default.t2[utf8_binary#x,utf8_binary_lcase#x] parquet -- !query @@ -150,39 +150,39 @@ DropTable false, false -- !query -create table t1 (c1 struct) USING PARQUET +create table t1 (c1 struct) USING PARQUET -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query -INSERT INTO t1 VALUES (named_struct('ucs_basic', 'aaa', 'ucs_basic_lcase', 'aaa')) +INSERT INTO t1 VALUES (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] -+- Project [named_struct(ucs_basic, col1#x.ucs_basic, ucs_basic_lcase, cast(col1#x.ucs_basic_lcase as string COLLATE UCS_BASIC_LCASE)) AS c1#x] ++- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_binary_lcase, cast(col1#x.utf8_binary_lcase as string COLLATE UTF8_BINARY_LCASE)) AS c1#x] +- LocalRelation [col1#x] -- !query -INSERT INTO t1 VALUES (named_struct('ucs_basic', 'AAA', 'ucs_basic_lcase', 'AAA')) +INSERT INTO t1 VALUES (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] -+- Project [named_struct(ucs_basic, col1#x.ucs_basic, ucs_basic_lcase, cast(col1#x.ucs_basic_lcase as string COLLATE UCS_BASIC_LCASE)) AS c1#x] ++- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_binary_lcase, cast(col1#x.utf8_binary_lcase as string COLLATE UTF8_BINARY_LCASE)) AS c1#x] +- LocalRelation [col1#x] -- !query -select count(*) from t1 group by c1.ucs_basic +select count(*) from t1 group by c1.utf8_binary -- !query analysis -Aggregate [c1#x.ucs_basic], [count(1) AS count(1)#xL] +Aggregate [c1#x.utf8_binary], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 +- Relation spark_catalog.default.t1[c1#x] parquet -- !query -select count(*) from t1 group by c1.ucs_basic_lcase +select count(*) from t1 group by c1.utf8_binary_lcase -- !query analysis -Aggregate [c1#x.ucs_basic_lcase], [count(1) AS count(1)#xL] +Aggregate [c1#x.utf8_binary_lcase], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 +- Relation spark_catalog.default.t1[c1#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index 91a159b39aeea..af87f7a321c2d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -1,54 +1,54 @@ -- test cases for collation support -- Create a test table with data -create table t1(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet; +create table t1(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet; insert into t1 values('aaa', 'aaa'); insert into t1 values('AAA', 'AAA'); insert into t1 values('bbb', 'bbb'); insert into t1 values('BBB', 'BBB'); --- group by and count ucs_basic -select count(*) from t1 group by ucs_basic; +-- group by and count utf8_binary +select count(*) from t1 group by utf8_binary; --- group by and count ucs_basic_lcase -select count(*) from t1 group by ucs_basic_lcase; +-- group by and count utf8_binary_lcase +select count(*) from t1 group by utf8_binary_lcase; --- filter equal ucs_basic -select * from t1 where ucs_basic = 'aaa'; +-- filter equal utf8_binary +select * from t1 where utf8_binary = 'aaa'; --- filter equal ucs_basic_lcase -select * from t1 where ucs_basic_lcase = 'aaa' collate ucs_basic_lcase; +-- filter equal utf8_binary_lcase +select * from t1 where utf8_binary_lcase = 'aaa' collate utf8_binary_lcase; --- filter less then ucs_basic -select * from t1 where ucs_basic < 'bbb'; +-- filter less then utf8_binary +select * from t1 where utf8_binary < 'bbb'; --- filter less then ucs_basic_lcase -select * from t1 where ucs_basic_lcase < 'bbb' collate ucs_basic_lcase; +-- filter less then utf8_binary_lcase +select * from t1 where utf8_binary_lcase < 'bbb' collate utf8_binary_lcase; -- inner join -select l.ucs_basic, r.ucs_basic_lcase from t1 l join t1 r on l.ucs_basic_lcase = r.ucs_basic_lcase; +select l.utf8_binary, r.utf8_binary_lcase from t1 l join t1 r on l.utf8_binary_lcase = r.utf8_binary_lcase; -- create second table for anti-join -create table t2(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet; +create table t2(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet; insert into t2 values('aaa', 'aaa'); insert into t2 values('bbb', 'bbb'); -- anti-join on lcase -select * from t1 anti join t2 on t1.ucs_basic_lcase = t2.ucs_basic_lcase; +select * from t1 anti join t2 on t1.utf8_binary_lcase = t2.utf8_binary_lcase; drop table t2; drop table t1; -- create table with struct field -create table t1 (c1 struct) USING PARQUET; +create table t1 (c1 struct) USING PARQUET; -INSERT INTO t1 VALUES (named_struct('ucs_basic', 'aaa', 'ucs_basic_lcase', 'aaa')); -INSERT INTO t1 VALUES (named_struct('ucs_basic', 'AAA', 'ucs_basic_lcase', 'AAA')); +INSERT INTO t1 VALUES (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')); +INSERT INTO t1 VALUES (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')); --- aggregate against nested field ucs_basic -select count(*) from t1 group by c1.ucs_basic; +-- aggregate against nested field utf8_binary +select count(*) from t1 group by c1.utf8_binary; --- aggregate against nested field ucs_basic_lcase -select count(*) from t1 group by c1.ucs_basic_lcase; +-- aggregate against nested field utf8_binary_lcase +select count(*) from t1 group by c1.utf8_binary_lcase; drop table t1; diff --git a/sql/core/src/test/resources/sql-tests/results/collations.sql.out b/sql/core/src/test/resources/sql-tests/results/collations.sql.out index 7814ae59173dd..d6e41bd2b0c97 100644 --- a/sql/core/src/test/resources/sql-tests/results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/collations.sql.out @@ -1,6 +1,6 @@ -- Automatically generated by SQLQueryTestSuite -- !query -create table t1(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet +create table t1(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet -- !query schema struct<> -- !query output @@ -40,7 +40,7 @@ struct<> -- !query -select count(*) from t1 group by ucs_basic +select count(*) from t1 group by utf8_binary -- !query schema struct -- !query output @@ -51,7 +51,7 @@ struct -- !query -select count(*) from t1 group by ucs_basic_lcase +select count(*) from t1 group by utf8_binary_lcase -- !query schema struct -- !query output @@ -60,26 +60,26 @@ struct -- !query -select * from t1 where ucs_basic = 'aaa' +select * from t1 where utf8_binary = 'aaa' -- !query schema -struct +struct -- !query output aaa aaa -- !query -select * from t1 where ucs_basic_lcase = 'aaa' collate ucs_basic_lcase +select * from t1 where utf8_binary_lcase = 'aaa' collate utf8_binary_lcase -- !query schema -struct +struct -- !query output AAA AAA aaa aaa -- !query -select * from t1 where ucs_basic < 'bbb' +select * from t1 where utf8_binary < 'bbb' -- !query schema -struct +struct -- !query output AAA AAA BBB BBB @@ -87,18 +87,18 @@ aaa aaa -- !query -select * from t1 where ucs_basic_lcase < 'bbb' collate ucs_basic_lcase +select * from t1 where utf8_binary_lcase < 'bbb' collate utf8_binary_lcase -- !query schema -struct +struct -- !query output AAA AAA aaa aaa -- !query -select l.ucs_basic, r.ucs_basic_lcase from t1 l join t1 r on l.ucs_basic_lcase = r.ucs_basic_lcase +select l.utf8_binary, r.utf8_binary_lcase from t1 l join t1 r on l.utf8_binary_lcase = r.utf8_binary_lcase -- !query schema -struct +struct -- !query output AAA AAA AAA aaa @@ -111,7 +111,7 @@ bbb bbb -- !query -create table t2(ucs_basic string collate ucs_basic, ucs_basic_lcase string collate ucs_basic_lcase) using parquet +create table t2(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet -- !query schema struct<> -- !query output @@ -135,9 +135,9 @@ struct<> -- !query -select * from t1 anti join t2 on t1.ucs_basic_lcase = t2.ucs_basic_lcase +select * from t1 anti join t2 on t1.utf8_binary_lcase = t2.utf8_binary_lcase -- !query schema -struct +struct -- !query output @@ -159,7 +159,7 @@ struct<> -- !query -create table t1 (c1 struct) USING PARQUET +create table t1 (c1 struct) USING PARQUET -- !query schema struct<> -- !query output @@ -167,7 +167,7 @@ struct<> -- !query -INSERT INTO t1 VALUES (named_struct('ucs_basic', 'aaa', 'ucs_basic_lcase', 'aaa')) +INSERT INTO t1 VALUES (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')) -- !query schema struct<> -- !query output @@ -175,7 +175,7 @@ struct<> -- !query -INSERT INTO t1 VALUES (named_struct('ucs_basic', 'AAA', 'ucs_basic_lcase', 'AAA')) +INSERT INTO t1 VALUES (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')) -- !query schema struct<> -- !query output @@ -183,7 +183,7 @@ struct<> -- !query -select count(*) from t1 group by c1.ucs_basic +select count(*) from t1 group by c1.utf8_binary -- !query schema struct -- !query output @@ -192,7 +192,7 @@ struct -- !query -select count(*) from t1 group by c1.ucs_basic_lcase +select count(*) from t1 group by c1.utf8_binary_lcase -- !query schema struct -- !query output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index 6f183e60f5895..2beb4b27004d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -36,7 +36,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { protected val v2Source = classOf[FakeV2ProviderWithCustomSchema].getName test("collate returns proper type") { - Seq("ucs_basic", "ucs_basic_lcase", "unicode", "unicode_ci").foreach { collationName => + Seq("utf8_binary", "utf8_binary_lcase", "unicode", "unicode_ci").foreach { collationName => checkAnswer(sql(s"select 'aaa' collate $collationName"), Row("aaa")) val collationId = CollationFactory.collationNameToId(collationName) assert(sql(s"select 'aaa' collate $collationName").schema(0).dataType @@ -45,7 +45,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } test("collation name is case insensitive") { - Seq("uCs_BasIc", "uCs_baSic_Lcase", "uNicOde", "UNICODE_ci").foreach { collationName => + Seq("uTf8_BiNaRy", "uTf8_BiNaRy_Lcase", "uNicOde", "UNICODE_ci").foreach { collationName => checkAnswer(sql(s"select 'aaa' collate $collationName"), Row("aaa")) val collationId = CollationFactory.collationNameToId(collationName) assert(sql(s"select 'aaa' collate $collationName").schema(0).dataType @@ -54,15 +54,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } test("collation expression returns name of collation") { - Seq("ucs_basic", "ucs_basic_lcase", "unicode", "unicode_ci").foreach { collationName => + Seq("utf8_binary", "utf8_binary_lcase", "unicode", "unicode_ci").foreach { collationName => checkAnswer( sql(s"select collation('aaa' collate $collationName)"), Row(collationName.toUpperCase())) } } test("collate function syntax") { - assert(sql(s"select collate('aaa', 'ucs_basic')").schema(0).dataType == StringType(0)) - assert(sql(s"select collate('aaa', 'ucs_basic_lcase')").schema(0).dataType == StringType(1)) + assert(sql(s"select collate('aaa', 'utf8_binary')").schema(0).dataType == StringType(0)) + assert(sql(s"select collate('aaa', 'utf8_binary_lcase')").schema(0).dataType == StringType(1)) } test("collate function syntax invalid arg count") { @@ -113,7 +113,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("collate function invalid input data type") { checkError( - exception = intercept[ExtendedAnalysisException] { sql(s"select collate(1, 'UCS_BASIC')") }, + exception = intercept[ExtendedAnalysisException] { sql(s"select collate(1, 'UTF8_BINARY')") }, errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = "42K09", parameters = Map( @@ -123,28 +123,28 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { "inputType" -> "\"INT\"", "requiredType" -> "\"STRING\""), context = ExpectedContext( - fragment = s"collate(1, 'UCS_BASIC')", start = 7, stop = 29)) + fragment = s"collate(1, 'UTF8_BINARY')", start = 7, stop = 31)) } test("collation expression returns default collation") { - checkAnswer(sql(s"select collation('aaa')"), Row("UCS_BASIC")) + checkAnswer(sql(s"select collation('aaa')"), Row("UTF8_BINARY")) } test("invalid collation name throws exception") { checkError( - exception = intercept[SparkException] { sql("select 'aaa' collate UCS_BASIS") }, + exception = intercept[SparkException] { sql("select 'aaa' collate UTF8_BS") }, errorClass = "COLLATION_INVALID_NAME", sqlState = "42704", - parameters = Map("proposal" -> "UCS_BASIC", "collationName" -> "UCS_BASIS")) + parameters = Map("proposal" -> "UTF8_BINARY", "collationName" -> "UTF8_BS")) } test("equality check respects collation") { Seq( - ("ucs_basic", "aaa", "AAA", false), - ("ucs_basic", "aaa", "aaa", true), - ("ucs_basic_lcase", "aaa", "aaa", true), - ("ucs_basic_lcase", "aaa", "AAA", true), - ("ucs_basic_lcase", "aaa", "bbb", false), + ("utf8_binary", "aaa", "AAA", false), + ("utf8_binary", "aaa", "aaa", true), + ("utf8_binary_lcase", "aaa", "aaa", true), + ("utf8_binary_lcase", "aaa", "AAA", true), + ("utf8_binary_lcase", "aaa", "bbb", false), ("unicode", "aaa", "aaa", true), ("unicode", "aaa", "AAA", false), ("unicode_CI", "aaa", "aaa", true), @@ -163,12 +163,12 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("comparisons respect collation") { Seq( - ("ucs_basic", "AAA", "aaa", true), - ("ucs_basic", "aaa", "aaa", false), - ("ucs_basic", "aaa", "BBB", false), - ("ucs_basic_lcase", "aaa", "aaa", false), - ("ucs_basic_lcase", "AAA", "aaa", false), - ("ucs_basic_lcase", "aaa", "bbb", true), + ("utf8_binary", "AAA", "aaa", true), + ("utf8_binary", "aaa", "aaa", false), + ("utf8_binary", "aaa", "BBB", false), + ("utf8_binary_lcase", "aaa", "aaa", false), + ("utf8_binary_lcase", "AAA", "aaa", false), + ("utf8_binary_lcase", "aaa", "bbb", true), ("unicode", "aaa", "aaa", false), ("unicode", "aaa", "AAA", true), ("unicode", "aaa", "BBB", true), @@ -252,15 +252,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("Support contains string expression with Collation") { // Supported collations val checks = Seq( - CollationTestCase("", "", "UCS_BASIC", true), - CollationTestCase("c", "", "UCS_BASIC", true), - CollationTestCase("", "c", "UCS_BASIC", false), - CollationTestCase("abcde", "c", "UCS_BASIC", true), - CollationTestCase("abcde", "C", "UCS_BASIC", false), - CollationTestCase("abcde", "bcd", "UCS_BASIC", true), - CollationTestCase("abcde", "BCD", "UCS_BASIC", false), - CollationTestCase("abcde", "fgh", "UCS_BASIC", false), - CollationTestCase("abcde", "FGH", "UCS_BASIC", false), + CollationTestCase("", "", "UTF8_BINARY", true), + CollationTestCase("c", "", "UTF8_BINARY", true), + CollationTestCase("", "c", "UTF8_BINARY", false), + CollationTestCase("abcde", "c", "UTF8_BINARY", true), + CollationTestCase("abcde", "C", "UTF8_BINARY", false), + CollationTestCase("abcde", "bcd", "UTF8_BINARY", true), + CollationTestCase("abcde", "BCD", "UTF8_BINARY", false), + CollationTestCase("abcde", "fgh", "UTF8_BINARY", false), + CollationTestCase("abcde", "FGH", "UTF8_BINARY", false), CollationTestCase("", "", "UNICODE", true), CollationTestCase("c", "", "UNICODE", true), CollationTestCase("", "c", "UNICODE", false), @@ -270,15 +270,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { CollationTestCase("abcde", "BCD", "UNICODE", false), CollationTestCase("abcde", "fgh", "UNICODE", false), CollationTestCase("abcde", "FGH", "UNICODE", false), - CollationTestCase("", "", "UCS_BASIC_LCASE", true), - CollationTestCase("c", "", "UCS_BASIC_LCASE", true), - CollationTestCase("", "c", "UCS_BASIC_LCASE", false), - CollationTestCase("abcde", "c", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "C", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "bcd", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "BCD", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "fgh", "UCS_BASIC_LCASE", false), - CollationTestCase("abcde", "FGH", "UCS_BASIC_LCASE", false), + CollationTestCase("", "", "UTF8_BINARY_LCASE", true), + CollationTestCase("c", "", "UTF8_BINARY_LCASE", true), + CollationTestCase("", "c", "UTF8_BINARY_LCASE", false), + CollationTestCase("abcde", "c", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "C", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "bcd", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "BCD", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "fgh", "UTF8_BINARY_LCASE", false), + CollationTestCase("abcde", "FGH", "UTF8_BINARY_LCASE", false), CollationTestCase("", "", "UNICODE_CI", true), CollationTestCase("c", "", "UNICODE_CI", true), CollationTestCase("", "c", "UNICODE_CI", false), @@ -298,15 +298,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("Support startsWith string expression with Collation") { // Supported collations val checks = Seq( - CollationTestCase("", "", "UCS_BASIC", true), - CollationTestCase("c", "", "UCS_BASIC", true), - CollationTestCase("", "c", "UCS_BASIC", false), - CollationTestCase("abcde", "a", "UCS_BASIC", true), - CollationTestCase("abcde", "A", "UCS_BASIC", false), - CollationTestCase("abcde", "abc", "UCS_BASIC", true), - CollationTestCase("abcde", "ABC", "UCS_BASIC", false), - CollationTestCase("abcde", "bcd", "UCS_BASIC", false), - CollationTestCase("abcde", "BCD", "UCS_BASIC", false), + CollationTestCase("", "", "UTF8_BINARY", true), + CollationTestCase("c", "", "UTF8_BINARY", true), + CollationTestCase("", "c", "UTF8_BINARY", false), + CollationTestCase("abcde", "a", "UTF8_BINARY", true), + CollationTestCase("abcde", "A", "UTF8_BINARY", false), + CollationTestCase("abcde", "abc", "UTF8_BINARY", true), + CollationTestCase("abcde", "ABC", "UTF8_BINARY", false), + CollationTestCase("abcde", "bcd", "UTF8_BINARY", false), + CollationTestCase("abcde", "BCD", "UTF8_BINARY", false), CollationTestCase("", "", "UNICODE", true), CollationTestCase("c", "", "UNICODE", true), CollationTestCase("", "c", "UNICODE", false), @@ -316,15 +316,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { CollationTestCase("abcde", "ABC", "UNICODE", false), CollationTestCase("abcde", "bcd", "UNICODE", false), CollationTestCase("abcde", "BCD", "UNICODE", false), - CollationTestCase("", "", "UCS_BASIC_LCASE", true), - CollationTestCase("c", "", "UCS_BASIC_LCASE", true), - CollationTestCase("", "c", "UCS_BASIC_LCASE", false), - CollationTestCase("abcde", "a", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "A", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "abc", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "ABC", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "bcd", "UCS_BASIC_LCASE", false), - CollationTestCase("abcde", "BCD", "UCS_BASIC_LCASE", false), + CollationTestCase("", "", "UTF8_BINARY_LCASE", true), + CollationTestCase("c", "", "UTF8_BINARY_LCASE", true), + CollationTestCase("", "c", "UTF8_BINARY_LCASE", false), + CollationTestCase("abcde", "a", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "A", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "abc", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "ABC", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "bcd", "UTF8_BINARY_LCASE", false), + CollationTestCase("abcde", "BCD", "UTF8_BINARY_LCASE", false), CollationTestCase("", "", "UNICODE_CI", true), CollationTestCase("c", "", "UNICODE_CI", true), CollationTestCase("", "c", "UNICODE_CI", false), @@ -344,15 +344,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("Support endsWith string expression with Collation") { // Supported collations val checks = Seq( - CollationTestCase("", "", "UCS_BASIC", true), - CollationTestCase("c", "", "UCS_BASIC", true), - CollationTestCase("", "c", "UCS_BASIC", false), - CollationTestCase("abcde", "e", "UCS_BASIC", true), - CollationTestCase("abcde", "E", "UCS_BASIC", false), - CollationTestCase("abcde", "cde", "UCS_BASIC", true), - CollationTestCase("abcde", "CDE", "UCS_BASIC", false), - CollationTestCase("abcde", "bcd", "UCS_BASIC", false), - CollationTestCase("abcde", "BCD", "UCS_BASIC", false), + CollationTestCase("", "", "UTF8_BINARY", true), + CollationTestCase("c", "", "UTF8_BINARY", true), + CollationTestCase("", "c", "UTF8_BINARY", false), + CollationTestCase("abcde", "e", "UTF8_BINARY", true), + CollationTestCase("abcde", "E", "UTF8_BINARY", false), + CollationTestCase("abcde", "cde", "UTF8_BINARY", true), + CollationTestCase("abcde", "CDE", "UTF8_BINARY", false), + CollationTestCase("abcde", "bcd", "UTF8_BINARY", false), + CollationTestCase("abcde", "BCD", "UTF8_BINARY", false), CollationTestCase("", "", "UNICODE", true), CollationTestCase("c", "", "UNICODE", true), CollationTestCase("", "c", "UNICODE", false), @@ -362,15 +362,15 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { CollationTestCase("abcde", "CDE", "UNICODE", false), CollationTestCase("abcde", "bcd", "UNICODE", false), CollationTestCase("abcde", "BCD", "UNICODE", false), - CollationTestCase("", "", "UCS_BASIC_LCASE", true), - CollationTestCase("c", "", "UCS_BASIC_LCASE", true), - CollationTestCase("", "c", "UCS_BASIC_LCASE", false), - CollationTestCase("abcde", "e", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "E", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "cde", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "CDE", "UCS_BASIC_LCASE", true), - CollationTestCase("abcde", "bcd", "UCS_BASIC_LCASE", false), - CollationTestCase("abcde", "BCD", "UCS_BASIC_LCASE", false), + CollationTestCase("", "", "UTF8_BINARY_LCASE", true), + CollationTestCase("c", "", "UTF8_BINARY_LCASE", true), + CollationTestCase("", "c", "UTF8_BINARY_LCASE", false), + CollationTestCase("abcde", "e", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "E", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "cde", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "CDE", "UTF8_BINARY_LCASE", true), + CollationTestCase("abcde", "bcd", "UTF8_BINARY_LCASE", false), + CollationTestCase("abcde", "BCD", "UTF8_BINARY_LCASE", false), CollationTestCase("", "", "UNICODE_CI", true), CollationTestCase("c", "", "UNICODE_CI", true), CollationTestCase("", "c", "UNICODE_CI", false), @@ -389,12 +389,12 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("aggregates count respects collation") { Seq( - ("ucs_basic", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))), - ("ucs_basic", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), - ("ucs_basic", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), - ("ucs_basic_lcase", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), - ("ucs_basic_lcase", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))), - ("ucs_basic_lcase", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), + ("utf8_binary", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))), + ("utf8_binary", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), + ("utf8_binary", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), + ("utf8_binary_lcase", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), + ("utf8_binary_lcase", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))), + ("utf8_binary_lcase", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), ("unicode", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))), ("unicode", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), ("unicode", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), @@ -420,9 +420,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val tableNameBinary = "T_BINARY" withTable(tableNameNonBinary) { withTable(tableNameBinary) { - sql(s"CREATE TABLE $tableNameNonBinary (c STRING COLLATE UCS_BASIC_LCASE) USING PARQUET") + sql(s"CREATE TABLE $tableNameNonBinary (c STRING COLLATE UTF8_BINARY_LCASE) USING PARQUET") sql(s"INSERT INTO $tableNameNonBinary VALUES ('aaa')") - sql(s"CREATE TABLE $tableNameBinary (c STRING COLLATE UCS_BASIC) USING PARQUET") + sql(s"CREATE TABLE $tableNameBinary (c STRING COLLATE UTF8_BINARY) USING PARQUET") sql(s"INSERT INTO $tableNameBinary VALUES ('aaa')") val dfNonBinary = sql(s"SELECT COUNT(*), c FROM $tableNameNonBinary GROUP BY c") @@ -450,7 +450,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("create table with collation") { val tableName = "parquet_dummy_tbl" - val collationName = "UCS_BASIC_LCASE" + val collationName = "UTF8_BINARY_LCASE" val collationId = CollationFactory.collationNameToId(collationName) withTable(tableName) { @@ -470,7 +470,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("create table with collations inside a struct") { val tableName = "struct_collation_tbl" - val collationName = "UCS_BASIC_LCASE" + val collationName = "UTF8_BINARY_LCASE" val collationId = CollationFactory.collationNameToId(collationName) withTable(tableName) { @@ -492,8 +492,8 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("add collated column with alter table") { val tableName = "alter_column_tbl" - val defaultCollation = "UCS_BASIC" - val collationName = "UCS_BASIC_LCASE" + val defaultCollation = "UTF8_BINARY" + val collationName = "UTF8_BINARY_LCASE" val collationId = CollationFactory.collationNameToId(collationName) withTable(tableName) { @@ -526,7 +526,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("create v2 table with collation column") { val tableName = "testcat.table_name" - val collationName = "UCS_BASIC_LCASE" + val collationName = "UTF8_BINARY_LCASE" val collationId = CollationFactory.collationNameToId(collationName) withTable(tableName) { @@ -590,7 +590,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val schema = StructType(StructField( "col", - StringType(CollationFactory.collationNameToId("UCS_BASIC_LCASE"))) :: Nil) + StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE"))) :: Nil) val df = spark.createDataFrame(sparkContext.parallelize(in), schema) df.repartition(10, df.col("col")).foreachPartition( @@ -609,7 +609,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val schema = StructType(StructField( "col_non_binary", - StringType(CollationFactory.collationNameToId("UCS_BASIC_LCASE"))) :: + StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE"))) :: StructField("col_binary", StringType) :: Nil) val df1 = spark.createDataFrame(sparkContext.parallelize(in), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 47e1f37d4be52..5f144d42649a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -1250,7 +1250,7 @@ class FileBasedDataSourceSuite extends QueryTest test("disable filter pushdown for collated strings") { Seq("parquet").foreach { format => withTempPath { path => - val collation = "'UCS_BASIC_LCASE'" + val collation = "'UTF8_BINARY_LCASE'" val df = sql( s"""SELECT | COLLATE(c, $collation) as c1, From 4b0d1085a7825be2a36d5d3d2513b735600a2d24 Mon Sep 17 00:00:00 2001 From: Stevo Mitric Date: Mon, 11 Mar 2024 19:18:30 +0500 Subject: [PATCH 28/50] [SPARK-47313][SQL] Added scala.MatchError handling inside QueryExecution.toInternalError ### What changes were proposed in this pull request? Created `isInternalError` function that marks `java.lang.NullPointerException`, `java.lang.AssertionError` and newly added `scala.MatchError` as internal errors. ### Why are the changes needed? Additional error coverage and code refactoring. ### Does this PR introduce _any_ user-facing change? No ### Was this patch authored or co-authored using generative AI tooling? No Closes #45438 from stevomitric/stevomitric/match-error-handling. Authored-by: Stevo Mitric Signed-off-by: Max Gekk --- .../spark/sql/execution/QueryExecution.scala | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) 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 a2cfad800e006..3034716526e1b 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 @@ -540,20 +540,31 @@ object QueryExecution { } /** - * Converts asserts, null pointer exceptions to internal errors. + * Marks null pointer exceptions, asserts and scala match errors as internal errors */ - private[sql] def toInternalError(msg: String, e: Throwable): Throwable = e match { - case e @ (_: java.lang.NullPointerException | _: java.lang.AssertionError) => + private[sql] def isInternalError(e: Throwable): Boolean = e match { + case _: java.lang.NullPointerException => true + case _: java.lang.AssertionError => true + case _: scala.MatchError => true + case _ => false + } + + /** + * Converts marked exceptions from [[isInternalError]] to internal errors. + */ + private[sql] def toInternalError(msg: String, e: Throwable): Throwable = { + if (isInternalError(e)) { SparkException.internalError( msg + " You hit a bug in Spark or the Spark plugins you use. Please, report this bug " + "to the corresponding communities or vendors, and provide the full stack trace.", e) - case e: Throwable => + } else { e + } } /** - * Catches asserts, null pointer exceptions, and converts them to internal errors. + * Catches marked exceptions from [[isInternalError]], and converts them to internal errors. */ private[sql] def withInternalError[T](msg: String)(block: => T): T = { try { From 10be0321577513b442cabc84269d402cffb3c73a Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Mon, 11 Mar 2024 19:33:05 +0500 Subject: [PATCH 29/50] [SPARK-47255][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-9] ### What changes were proposed in this pull request? In the PR, I propose to assign the proper names to the legacy error classes _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-9], and modify tests in testing suites to reflect these changes and use checkError() function. Also this PR improves the error messages. ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? Error _LEGACY_ERROR_TEMP_3249 is tested by running the modified test suite: `$ build/sbt "catalyst/testOnly *RowJsonSuite"` Errors _LEGACY_ERROR_TEMP_323[6-7] and _LEGACY_ERROR_TEMP_324[7-8] are tested by running the modified test suite: `$ build/sbt "catalyst/testOnly *CSVExprUtilsSuite"` Golden files are regenerated using the: `$ SPARK_GENERATE_GOLDEN_FILES=1` `$ build/sbt "core/testOnly *SparkThrowableSuite"` ### Was this patch authored or co-authored using generative AI tooling? No Closes #45423 from miland-db/miland-db/miland-legacy-error-class. Authored-by: Milan Dankovic Signed-off-by: Max Gekk --- .../main/resources/error/error-classes.json | 59 +++++++++++-------- ...ons-invalid-delimiter-value-error-class.md | 49 +++++++++++++++ docs/sql-error-conditions.md | 14 +++++ .../main/scala/org/apache/spark/sql/Row.scala | 9 +-- .../spark/sql/catalyst/csv/CSVExprUtils.scala | 13 ++-- .../org/apache/spark/sql/RowJsonSuite.scala | 11 ++-- .../sql/catalyst/csv/CSVExprUtilsSuite.scala | 10 ++-- 7 files changed, 123 insertions(+), 42 deletions(-) create mode 100644 docs/sql-error-conditions-invalid-delimiter-value-error-class.md diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 9717ff2ed49c4..afe81b8e9bea7 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1277,6 +1277,12 @@ ], "sqlState" : "58030" }, + "FAILED_ROW_TO_JSON" : { + "message" : [ + "Failed to convert the row value of the class to the target SQL type in the JSON format." + ], + "sqlState" : "2203G" + }, "FIELDS_ALREADY_EXISTS" : { "message" : [ "Cannot column, because already exists in ." @@ -1862,6 +1868,34 @@ }, "sqlState" : "42623" }, + "INVALID_DELIMITER_VALUE" : { + "message" : [ + "Invalid value for delimiter." + ], + "subClass" : { + "DELIMITER_LONGER_THAN_EXPECTED" : { + "message" : [ + "Delimiter cannot be more than one character: ." + ] + }, + "EMPTY_STRING" : { + "message" : [ + "Delimiter cannot be empty string." + ] + }, + "SINGLE_BACKSLASH" : { + "message" : [ + "Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter." + ] + }, + "UNSUPPORTED_SPECIAL_CHARACTER" : { + "message" : [ + "Unsupported special character for delimiter: ." + ] + } + }, + "sqlState" : "42602" + }, "INVALID_DRIVER_MEMORY" : { "message" : [ "System memory must be at least .", @@ -7767,16 +7801,6 @@ "The numbers of zipped arrays and field names should be the same" ] }, - "_LEGACY_ERROR_TEMP_3236" : { - "message" : [ - "Unsupported special character for delimiter: " - ] - }, - "_LEGACY_ERROR_TEMP_3237" : { - "message" : [ - "Delimiter cannot be more than one character: " - ] - }, "_LEGACY_ERROR_TEMP_3238" : { "message" : [ "Failed to convert value (class of ) in type
to XML." @@ -7822,21 +7846,6 @@ "Failed to parse a value for data type ." ] }, - "_LEGACY_ERROR_TEMP_3247" : { - "message" : [ - "Delimiter cannot be empty string" - ] - }, - "_LEGACY_ERROR_TEMP_3248" : { - "message" : [ - "Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter." - ] - }, - "_LEGACY_ERROR_TEMP_3249" : { - "message" : [ - "Failed to convert value (class of }) with the type of to JSON." - ] - }, "_LEGACY_ERROR_TEMP_3250" : { "message" : [ "Failed to convert the JSON string '' to a field." diff --git a/docs/sql-error-conditions-invalid-delimiter-value-error-class.md b/docs/sql-error-conditions-invalid-delimiter-value-error-class.md new file mode 100644 index 0000000000000..815fe78bce945 --- /dev/null +++ b/docs/sql-error-conditions-invalid-delimiter-value-error-class.md @@ -0,0 +1,49 @@ +--- +layout: global +title: INVALID_DELIMITER_VALUE error class +displayTitle: INVALID_DELIMITER_VALUE error class +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- + + + +[SQLSTATE: 42602](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Invalid value for delimiter. + +This error class has the following derived error classes: + +## DELIMITER_LONGER_THAN_EXPECTED + +Delimiter cannot be more than one character: ``. + +## EMPTY_STRING + +Delimiter cannot be empty string. + +## SINGLE_BACKSLASH + +Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter. + +## UNSUPPORTED_SPECIAL_CHARACTER + +Unsupported special character for delimiter: ``. + + diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 0be75cde968fb..0695ed28b7fc8 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -756,6 +756,12 @@ SQLSTATE: 58030 Failed to rename temp file `` to `` as FileSystem.rename returned false. +### FAILED_ROW_TO_JSON + +[SQLSTATE: 2203G](sql-error-conditions-sqlstates.html#class-22-data-exception) + +Failed to convert the row value `` of the class `` to the target SQL type `` in the JSON format. + ### FIELDS_ALREADY_EXISTS [SQLSTATE: 42710](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) @@ -1124,6 +1130,14 @@ Failed to execute `` command because the destination column or variab For more details see [INVALID_DEFAULT_VALUE](sql-error-conditions-invalid-default-value-error-class.html) +### [INVALID_DELIMITER_VALUE](sql-error-conditions-invalid-delimiter-value-error-class.html) + +[SQLSTATE: 42602](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Invalid value for delimiter. + +For more details see [INVALID_DELIMITER_VALUE](sql-error-conditions-invalid-delimiter-value-error-class.html) + ### INVALID_DRIVER_MEMORY SQLSTATE: F0000 diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala index 3d49dbf80c7d1..0c065dd4d4baa 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala @@ -34,6 +34,7 @@ import org.apache.spark.annotation.{Stable, Unstable} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.util.{DateFormatter, SparkDateTimeUtils, TimestampFormatter, UDTUtils} import org.apache.spark.sql.errors.DataTypeErrors +import org.apache.spark.sql.errors.DataTypeErrors.{toSQLType, toSQLValue} import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -611,11 +612,11 @@ trait Row extends Serializable { case (v: Any, udt: UserDefinedType[Any @unchecked]) => toJson(UDTUtils.toRow(v, udt), udt.sqlType) case _ => throw new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_3249", + errorClass = "FAILED_ROW_TO_JSON", messageParameters = Map( - "value" -> value.toString, - "valueClass" -> value.getClass.toString, - "dataType" -> dataType.toString) + "value" -> toSQLValue(value.toString), + "class" -> value.getClass.toString, + "sqlType" -> toSQLType(dataType.toString)) ) } toJson(this, schema) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala index 60113d6b3e125..62638d70dd904 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala @@ -68,8 +68,9 @@ object CSVExprUtils { @throws[SparkIllegalArgumentException] def toChar(str: String): Char = { (str: Seq[Char]) match { - case Seq() => throw new SparkIllegalArgumentException("_LEGACY_ERROR_TEMP_3247") - case Seq('\\') => throw new SparkIllegalArgumentException("_LEGACY_ERROR_TEMP_3248") + case Seq() => throw new SparkIllegalArgumentException("INVALID_DELIMITER_VALUE.EMPTY_STRING") + case Seq('\\') => + throw new SparkIllegalArgumentException("INVALID_DELIMITER_VALUE.SINGLE_BACKSLASH") case Seq(c) => c case Seq('\\', 't') => '\t' case Seq('\\', 'r') => '\r' @@ -82,10 +83,14 @@ object CSVExprUtils { case _ if str == "\u0000" => '\u0000' case Seq('\\', _) => throw new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_3236", messageParameters = Map("str" -> str)) + errorClass = + "INVALID_DELIMITER_VALUE.UNSUPPORTED_SPECIAL_CHARACTER", + messageParameters = Map("str" -> str)) case _ => throw new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_3237", messageParameters = Map("str" -> str)) + errorClass = + "INVALID_DELIMITER_VALUE.DELIMITER_LONGER_THAN_EXPECTED", + messageParameters = Map("str" -> str)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala index e5914d4a88d26..3e72dc7da24b7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala @@ -24,6 +24,7 @@ import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObje import org.apache.spark.{SparkFunSuite, SparkIllegalArgumentException} import org.apache.spark.sql.catalyst.encoders.{ExamplePoint, ExamplePointUDT} import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.errors.DataTypeErrors.{toSQLType, toSQLValue} import org.apache.spark.sql.types._ /** @@ -135,10 +136,12 @@ class RowJsonSuite extends SparkFunSuite { new StructType().add("a", ObjectType(classOf[(Int, Int)]))) row.jsonValue }, - errorClass = "_LEGACY_ERROR_TEMP_3249", + errorClass = "FAILED_ROW_TO_JSON", parameters = Map( - "value" -> "(1,2)", - "valueClass" -> "class scala.Tuple2$mcII$sp", - "dataType" -> "ObjectType(class scala.Tuple2)")) + "value" -> toSQLValue("(1,2)"), + "class" -> "class scala.Tuple2$mcII$sp", + "sqlType" -> toSQLType("ObjectType(class scala.Tuple2)") + ) + ) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index b2e4cdfeafe2d..2e94c723a6f26 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -38,7 +38,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("ab") }, - errorClass = "_LEGACY_ERROR_TEMP_3237", + errorClass = "INVALID_DELIMITER_VALUE.DELIMITER_LONGER_THAN_EXPECTED", parameters = Map("str" -> "ab")) } @@ -47,7 +47,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("""\1""") }, - errorClass = "_LEGACY_ERROR_TEMP_3236", + errorClass = "INVALID_DELIMITER_VALUE.UNSUPPORTED_SPECIAL_CHARACTER", parameters = Map("str" -> """\1""")) } @@ -56,7 +56,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("""\""") }, - errorClass = "_LEGACY_ERROR_TEMP_3248", + errorClass = "INVALID_DELIMITER_VALUE.SINGLE_BACKSLASH", parameters = Map.empty) } @@ -65,7 +65,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { exception = intercept[SparkIllegalArgumentException]{ CSVExprUtils.toChar("") }, - errorClass = "_LEGACY_ERROR_TEMP_3247", + errorClass = "INVALID_DELIMITER_VALUE.EMPTY_STRING", parameters = Map.empty) } @@ -76,7 +76,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { // backslash, then tab ("""\\t""", Some("""\t"""), None), // invalid special character (dot) - ("""\.""", None, Some("_LEGACY_ERROR_TEMP_3236")), + ("""\.""", None, Some("INVALID_DELIMITER_VALUE.UNSUPPORTED_SPECIAL_CHARACTER")), // backslash, then dot ("""\\.""", Some("""\."""), None), // nothing special, just straight conversion From 610840e27e2e05afafa6cee2fa6642c617e25669 Mon Sep 17 00:00:00 2001 From: cashmand Date: Mon, 11 Mar 2024 10:57:08 -0700 Subject: [PATCH 30/50] [SPARK-45827][SQL][FOLLOWUP] Fix for collation ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/45409 created a default allow-list of types for data sources. The intent was to only prevent creation of the two types that had already been prevented elsewhere in code, but the match expression matched `StringType`, which is an object representing the default collation, instead of the `StringType` class, which represents any collation. This PR fixes the issue. ### Why are the changes needed? Without it, the previous PR would be a breaking change for data sources that write StringType with a non-default collation. ### Does this PR introduce _any_ user-facing change? It reverts the previous unintentional user-facing change. ### How was this patch tested? Unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45463 from cashmand/SPARK-45827-followup. Authored-by: cashmand Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 2 +- .../execution/datasources/SaveIntoDataSourceCommandSuite.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 057a6c8bd9b3a..2e99a9afa3042 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -192,7 +192,7 @@ trait CreatableRelationProvider { case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) case BinaryType | BooleanType | ByteType | CharType(_) | DateType | _ : DecimalType | DoubleType | FloatType | IntegerType | LongType | NullType | ObjectType(_) | ShortType | - StringType | TimestampNTZType | TimestampType | VarcharType(_) => true + _: StringType | TimestampNTZType | TimestampType | VarcharType(_) => true case _ => false } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala index 6dcd228842423..51c9b960a8eab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -81,7 +81,8 @@ class SaveIntoDataSourceCommandSuite extends QueryTest with SharedSparkSession { options = Map()) val df = spark.range(1).selectExpr( - "cast('a' as binary) a", "true b", "cast(1 as byte) c", "1.23 d") + "cast('a' as binary) a", "true b", "cast(1 as byte) c", "1.23 d", "'abc'", + "'abc' COLLATE UTF8_BINARY_LCASE") dataSource.planForWriting(SaveMode.ErrorIfExists, df.logicalPlan) // Variant and Interval types are disallowed by default. From f97da163806277f373e803ebd29ea96de864041a Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 11 Mar 2024 10:59:37 -0700 Subject: [PATCH 31/50] [SPARK-45245][CONNECT][TESTS][FOLLOW-UP] Remove unneeded Matchers trait in the test ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/43023 that addresses a post-review comment. ### Why are the changes needed? It is unnecessary. It also matters with Scala compatibility so should better remove if unused. ### 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 #45459 from HyukjinKwon/SPARK-45245-folllowup. Lead-authored-by: Hyukjin Kwon Co-authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- .../apache/spark/api/python/PythonWorkerFactorySuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonWorkerFactorySuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonWorkerFactorySuite.scala index 34c10bd95ed75..359b76c87a2a5 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonWorkerFactorySuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonWorkerFactorySuite.scala @@ -25,15 +25,13 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future import scala.concurrent.duration._ -import org.scalatest.matchers.must.Matchers - import org.apache.spark.SharedSparkContext import org.apache.spark.SparkException import org.apache.spark.SparkFunSuite import org.apache.spark.util.ThreadUtils // Tests for PythonWorkerFactory. -class PythonWorkerFactorySuite extends SparkFunSuite with Matchers with SharedSparkContext { +class PythonWorkerFactorySuite extends SparkFunSuite with SharedSparkContext { test("createSimpleWorker() fails with a timeout error if worker does not connect back") { // It verifies that server side times out in accept(), if the worker does not connect back. From f40c693ad7fd5ee9936374278fc96511f2011e4f Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 11 Mar 2024 11:03:56 -0700 Subject: [PATCH 32/50] [SPARK-47339][BUILD] Upgrade checkStyle to `10.14.0` ### What changes were proposed in this pull request? The pr aims to upgrade checkStyle from `9.3` to `10.14.0`. ### Why are the changes needed? - In Spark 4.0, our JDK version has been upgraded to `JDK17` (no longer supporting `JDK1.8`), so this obstacle has been eliminated, and we can upgrade the version of `checkStyle` to the `10.x`. image https://checkstyle.sourceforge.io/index.html - The full release notes: v10.14.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.14.0 v10.13.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.13.0 v10.12.7: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.7 v10.12.6: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.6 v10.12.5: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.5 v10.12.4: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.4 v10.12.3: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.3 v10.12.2: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.2 v10.12.1: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.1 v10.12.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.12.0 v10.11.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.11.0 v10.10.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.10.0 v10.9.3: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.9.3 v10.9.2: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.9.2 v10.9.1: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.9.1 v10.9.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.9.0 v10.8.1: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.8.1 v10.8.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.8.0 v10.7.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.7.0 v10.6.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.6.0 v10.5.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.5.0 v10.4: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.4 v10.3.4: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.3.4 v10.3.3: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.3.3 v10.3.2: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.3.2 v10.3.1: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.3.1 v10.3: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.3 v10.2: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.2 v10.1: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.1 v10.0: https://checkstyle.sourceforge.io/releasenotes.html#Release_10.0 - The last upgrade happened 1.5 years ago: https://github.com/apache/spark/pull/37506 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test: ``` ./dev/lint-java ./dev/sbt-checkstyle ``` - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45451 from panbingkun/upgrade_checkstyle. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- .../src/main/java/org/apache/spark/launcher/SparkAppHandle.java | 2 +- pom.xml | 2 +- project/plugins.sbt | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java index afec270e2b11c..9fd693245aad2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java @@ -114,7 +114,7 @@ public boolean isFinal() { * * @since 1.6.0 */ - public interface Listener { + interface Listener { /** * Callback for changes in the handle's state. diff --git a/pom.xml b/pom.xml index 404f37be1b5ae..146ded53dd8da 100644 --- a/pom.xml +++ b/pom.xml @@ -3502,7 +3502,7 @@ --> com.puppycrawl.tools checkstyle - 9.3 + 10.14.0 diff --git a/project/plugins.sbt b/project/plugins.sbt index 628e1e6d89386..49e1afaa68549 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -20,7 +20,7 @@ addSbtPlugin("software.purpledragon" % "sbt-checkstyle-plugin" % "4.0.1") // sbt-checkstyle-plugin uses an old version of checkstyle. Match it to Maven's. // If you are changing the dependency setting for checkstyle plugin, // please check pom.xml in the root of the source tree too. -libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "9.3" +libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "10.14.0" // checkstyle uses guava 31.0.1-jre. libraryDependencies += "com.google.guava" % "guava" % "31.0.1-jre" From c1b9f28682d8948174a36aa7690df248fcd180dd Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Tue, 12 Mar 2024 08:47:08 +0900 Subject: [PATCH 33/50] [SPARK-47327][SQL] Fix thread safety issue in ICU Collator ### What changes were proposed in this pull request? Freezing the ICU collator upon creation. ### Why are the changes needed? In order to avoid multiple threads writing to the collation buffer during the generation of collation sort keys which then results in data corruption and an internal error. You can read more about collator thread safety [here](https://unicode-org.github.io/icu/userguide/icu/design.html#icu-threading-model-and-open-and-close-model) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unti test ### Was this patch authored or co-authored using generative AI tooling? no Closes #45436 from stefankandic/icuConcurrencyIssue. Authored-by: Stefan Kandic Signed-off-by: Hyukjin Kwon --- .../spark/sql/catalyst/util/CollationFactory.java | 2 ++ .../org/apache/spark/sql/CollationSuite.scala | 14 ++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java index 151a1c9ddbbec..2940900b974ad 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java @@ -138,11 +138,13 @@ public Collation( collationTable[2] = new Collation( "UNICODE", Collator.getInstance(ULocale.ROOT), "153.120.0.0", true); collationTable[2].collator.setStrength(Collator.TERTIARY); + collationTable[2].collator.freeze(); // UNICODE case-insensitive comparison (ROOT locale, in ICU + Secondary strength). collationTable[3] = new Collation( "UNICODE_CI", Collator.getInstance(ULocale.ROOT), "153.120.0.0", false); collationTable[3].collator.setStrength(Collator.SECONDARY); + collationTable[3].collator.freeze(); for (int i = 0; i < collationTable.length; i++) { collationNameToIdMap.put(collationTable[i].collationName, i); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index 2beb4b27004d5..f4d91a04753a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import scala.collection.immutable.Seq +import scala.collection.parallel.CollectionConverters.ImmutableIterableIsParallelizable import scala.jdk.CollectionConverters.MapHasAsJava import org.apache.spark.SparkException @@ -438,6 +439,19 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } + test("test concurrently generating collation keys") { + // generating ICU sort keys is not thread-safe by default so this should fail + // if we don't handle the concurrency properly on Collator level + + (0 to 10).foreach(_ => { + val collator = CollationFactory.fetchCollation("UNICODE").collator + + (0 to 100).par.foreach { _ => + collator.getCollationKey("aaa") + } + }) + } + test("text writing to parquet with collation enclosed with backticks") { withTempPath{ path => sql(s"select 'a' COLLATE `UNICODE`").write.parquet(path.getAbsolutePath) From f20b28e1005c90b95c5ae38988a824f0da91e9d2 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Tue, 12 Mar 2024 08:50:38 +0900 Subject: [PATCH 34/50] [SPARK-47341][CONNECT] Replace commands with relations in a few tests in SparkConnectClientSuite ### What changes were proposed in this pull request? A few [tests](https://github.com/apache/spark/blob/master/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala#L481-L527) in SparkConnectClientSuite attempt to test the result collection of a reattachable execution through the use of a SQL command. The SQL command, on a real server, is not executed eagerly (since it is a select command) and thus, is not entirely accurate. The test itself is non-problematic since a dummy server with dummy responses is used but a small improvement here would be to construct a relation rather than a command. ### Why are the changes needed? Although these tests are not problematic, we should ensure that the behavior of the tests is consistent with the actual behavior of real servers to avoid misleading developers. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The tests in org.apache.spark.sql.connect.client.SparkConnectClientSuite passed. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45460 from xi-db/SPARK-47341-fix-misleading-tests. Authored-by: Xi Lyu Signed-off-by: Hyukjin Kwon --- .../client/SparkConnectClientSuite.scala | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index b0c4564130d3a..5a43cf014bdc4 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -32,7 +32,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkException, SparkThrowable} import org.apache.spark.connect.proto -import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse, AnalyzePlanRequest, AnalyzePlanResponse, ArtifactStatusesRequest, ArtifactStatusesResponse, ExecutePlanRequest, ExecutePlanResponse, SparkConnectServiceGrpc} +import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse, AnalyzePlanRequest, AnalyzePlanResponse, ArtifactStatusesRequest, ArtifactStatusesResponse, ExecutePlanRequest, ExecutePlanResponse, Relation, SparkConnectServiceGrpc, SQL} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connect.common.config.ConnectCommon import org.apache.spark.sql.test.ConnectFunSuite @@ -478,6 +478,13 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { session.addArtifact(artifactFilePath.resolve("smallClassFile.class").toString) } + private def buildPlan(query: String): proto.Plan = { + proto.Plan + .newBuilder() + .setRoot(Relation.newBuilder().setSql(SQL.newBuilder().setQuery(query)).build()) + .build() + } + test("SPARK-45871: Client execute iterator.toSeq consumes the reattachable iterator") { startDummyServer(0) client = SparkConnectClient @@ -485,14 +492,9 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { .connectionString(s"sc://localhost:${server.getPort}") .enableReattachableExecute() .build() - val session = SparkSession.builder().client(client).create() - val cmd = session.newCommand(b => - b.setSqlCommand( - proto.SqlCommand - .newBuilder() - .setSql("select * from range(10000000)"))) - val plan = proto.Plan.newBuilder().setCommand(cmd) - val iter = client.execute(plan.build()) + + val plan = buildPlan("select * from range(10000000)") + val iter = client.execute(plan) val reattachableIter = ExecutePlanResponseReattachableIterator.fromIterator(iter) iter.toSeq @@ -512,14 +514,9 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { .connectionString(s"sc://localhost:${server.getPort}") .enableReattachableExecute() .build() - val session = SparkSession.builder().client(client).create() - val cmd = session.newCommand(b => - b.setSqlCommand( - proto.SqlCommand - .newBuilder() - .setSql("select * from range(10000000)"))) - val plan = proto.Plan.newBuilder().setCommand(cmd) - val iter = client.execute(plan.build()) + + val plan = buildPlan("select * from range(10000000)") + val iter = client.execute(plan) val reattachableIter = ExecutePlanResponseReattachableIterator.fromIterator(iter) iter.foreach(_ => ()) From 63c01c8036e8eb672bcca8bc2df994882a1b1727 Mon Sep 17 00:00:00 2001 From: Shujing Yang Date: Tue, 12 Mar 2024 09:24:36 +0900 Subject: [PATCH 35/50] [SPARK-47309][SQL][XML] Add schema inference unit tests ### What changes were proposed in this pull request? As titled. ### Why are the changes needed? Fix a bug. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #45411 from shujingyang-db/xml-inference-check. Authored-by: Shujing Yang Signed-off-by: Hyukjin Kwon --- .../datasources/xml/TestXmlData.scala | 310 ++++++++++++++++++ .../datasources/xml/XmlInferSchemaSuite.scala | 296 +++++++++++++++++ 2 files changed, 606 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestXmlData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestXmlData.scala index abcf8c7cdd726..704a02482ada8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestXmlData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/TestXmlData.scala @@ -68,4 +68,314 @@ private[xml] trait TestXmlData { f(dir) fs.setVerifyChecksum(true) } + + def primitiveFieldValueTypeConflict: Seq[String] = + """ + | 11 + | + | 1.1 + | true + | 13.1 + | str1 + | + |""".stripMargin :: + """ + | + | + | 21474836470.9 + | + | 12 + | + | true + |""".stripMargin :: + """ + | + | 21474836470 + | 92233720368547758070 + | 100 + | false + | str1 + | false + |""".stripMargin :: + """ + | + | 21474836570 + | 1.1 + | 21474836470 + | + | 92233720368547758070 + | + |""".stripMargin :: Nil + + def xmlNullStruct: Seq[String] = + """ + | + | 27.31.100.29 + | + | 1.abc.com + | UTF-8 + | + |""".stripMargin :: + """ + | + | 27.31.100.29 + | + |""".stripMargin :: + """ + | + | 27.31.100.29 + | + |""".stripMargin :: + """ + | + | 27.31.100.29 + | + |""".stripMargin :: Nil + + def complexFieldValueTypeConflict: Seq[String] = + """ + 11 + 1 + 2 + 3 + + + + """ :: + """ + + false + + + + + + """ :: + """ + + str + 4 + 5 + 6 + 7 + 8 + 9 + + + + """ :: + """ + + str1 + str2 + 33 + 7 + + true + + + str + + """ :: Nil + + def arrayElementTypeConflict: Seq[String] = + """ + | + | + | 1 + | 1.1 + | true + | + | + | + | + | + | + | + | + | + | + | + | 2 + | 3 + | 4 + | + | + | + | + | str + | + | + | + | + | 214748364700 + | + | + | 1 + | + | + |""".stripMargin :: + """ + | + | + | str + | + | + | 1 + | + | + |""".stripMargin :: + """ + | + | 1 + | 2 + | 3 + | + |""".stripMargin :: Nil + + def missingFields: Seq[String] = + """ + true + """ :: + """ + 21474836470 + """ :: + """ + 3344 + """ :: + """ + true + """ :: + """ + str + """ :: Nil + + // XML doesn't support array of arrays + // It only supports array of structs + def complexFieldAndType1: Seq[String] = + """ + | + | + | true + | 92233720368547758070 + | + | + | 4 + | 5 + | 6 + | str1 + | str2 + | + | str1 + | str2 + | 1 + | 2147483647 + | -2147483648 + | 21474836470 + | 9223372036854775807 + | -9223372036854775808 + | 922337203685477580700 + | -922337203685477580800 + | 1.2 + | 1.7976931348623157 + | 4.9E-324 + | 2.2250738585072014E-308 + | true + | false + | true + | + | + | + | true + | str1 + | + | + | false + | + | + | + | + | + | 123 + | + | + | str1str2 + | + | + | 123 + | + | + | 1.12.13.1 + | + | + | + |""".stripMargin :: Nil + + def complexFieldAndType2: Seq[String] = + """ + | + | + | + | 5 + | + | + | + | + | 67 + | + | + | 8 + | + | + | + | + | + | str1 + | + | + | + | + | + | + | + | str3 + | str33 + | + | + | str4 + | str11 + | + | + | + | + | + | + | + | 2 + | 3 + | + | + | + | + | + | + |""".stripMargin :: Nil + + def emptyRecords: Seq[String] = + """ + + """ :: + """ + + + + """ :: + """ + + + + + + + """ :: Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala new file mode 100644 index 0000000000000..697bd3d8b824f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlInferSchemaSuite.scala @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.xml + +import org.apache.spark.sql.{DataFrame, Encoders, QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{ + ArrayType, + BooleanType, + DecimalType, + DoubleType, + LongType, + StringType, + StructField, + StructType +} + +class XmlInferSchemaSuite extends QueryTest with SharedSparkSession with TestXmlData { + + val baseOptions = Map("rowTag" -> "ROW") + + def readData(xmlString: Seq[String], options: Map[String, String] = Map.empty): DataFrame = { + val dataset = spark.createDataset(spark.sparkContext.parallelize(xmlString))(Encoders.STRING) + spark.read.options(baseOptions ++ options).xml(dataset) + } + + // TODO: add tests for type widening + test("Type conflict in primitive field values") { + val xmlDF = readData(primitiveFieldValueTypeConflict, Map("nullValue" -> "")) + val expectedSchema = StructType( + StructField("num_bool", StringType, true) :: + StructField("num_num_1", LongType, true) :: + StructField("num_num_2", DoubleType, true) :: + StructField("num_num_3", DoubleType, true) :: + StructField("num_str", StringType, true) :: + StructField("str_bool", StringType, true) :: Nil + ) + val expectedAns = Row("true", 11L, null, 1.1, "13.1", "str1") :: + Row("12", null, 21474836470.9, null, null, "true") :: + Row("false", 21474836470L, 92233720368547758070d, 100, "str1", "false") :: + Row(null, 21474836570L, 1.1, 21474836470L, "92233720368547758070", null) :: Nil + assert(expectedSchema == xmlDF.schema) + checkAnswer(xmlDF, expectedAns) + } + + test("Type conflict in complex field values") { + val xmlDF = readData( + complexFieldValueTypeConflict, + Map("nullValue" -> "", "ignoreSurroundingSpaces" -> "true") + ) + // XML will merge an array and a singleton into an array + val expectedSchema = StructType( + StructField("array", ArrayType(LongType, true), true) :: + StructField("num_struct", StringType, true) :: + StructField("str_array", ArrayType(StringType), true) :: + StructField("struct", StructType(StructField("field", StringType, true) :: Nil), true) :: + StructField("struct_array", ArrayType(StringType), true) :: Nil + ) + + assert(expectedSchema === xmlDF.schema) + checkAnswer( + xmlDF, + Row(Seq(null), "11", Seq("1", "2", "3"), Row(null), Seq(null)) :: + Row(Seq(null), """false""", Seq(null), Row(null), Seq(null)) :: + Row(Seq(4, 5, 6), null, Seq("str"), Row(null), Seq("7", "8", "9")) :: + Row(Seq(7), null, Seq("str1", "str2", "33"), Row("str"), Seq("""true""")) :: + Nil + ) + } + + test("Type conflict in array elements") { + val xmlDF = + readData( + arrayElementTypeConflict, + Map("ignoreSurroundingSpaces" -> "true", "nullValue" -> "")) + + val expectedSchema = StructType( + StructField( + "array1", + ArrayType(StructType(StructField("element", ArrayType(StringType)) :: Nil), true), + true + ) :: + StructField( + "array2", + ArrayType(StructType(StructField("field", LongType, true) :: Nil), true), + true + ) :: + StructField("array3", ArrayType(StringType, true), true) :: Nil + ) + + assert(xmlDF.schema === expectedSchema) + checkAnswer( + xmlDF, + Row( + Seq( + Row(List("1", "1.1", "true", null, "", "")), + Row( + List( + """ + | 2 + | 3 + | 4 + | """.stripMargin, + """ + | str + | """.stripMargin + ) + ) + ), + Seq(Row(214748364700L), Row(1)), + null + ) :: + Row(null, null, Seq("""str""", """1""")) :: + Row(null, null, Seq("1", "2", "3")) :: Nil + ) + } + + test("Handling missing fields") { + val xmlDF = readData(missingFields) + + val expectedSchema = StructType( + StructField("a", BooleanType, true) :: + StructField("b", LongType, true) :: + StructField("c", ArrayType(LongType, true), true) :: + StructField("d", StructType(StructField("field", BooleanType, true) :: Nil), true) :: + StructField("e", StringType, true) :: Nil + ) + + assert(expectedSchema === xmlDF.schema) + + } + + test("Complex field and type inferring") { + val xmlDF = readData(complexFieldAndType1, Map("prefersDecimal" -> "true")) + val expectedSchema = StructType( + StructField( + "arrayOfArray1", + ArrayType(StructType(StructField("item", ArrayType(StringType, true)) :: Nil)), + true + ) :: + StructField( + "arrayOfArray2", + ArrayType(StructType(StructField("item", ArrayType(DecimalType(21, 1), true)) :: Nil), true) + ) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType(21, 0), true), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType, true), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType, true), true) :: + StructField("arrayOfInteger", ArrayType(LongType, true), true) :: + StructField("arrayOfLong", ArrayType(DecimalType(20, 0), true), true) :: + StructField("arrayOfNull", ArrayType(StringType, true), true) :: + StructField("arrayOfString", ArrayType(StringType, true), true) :: + StructField( + "arrayOfStruct", + ArrayType( + StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", StringType, true) :: + StructField("field3", StringType, true) :: Nil + ), + true + ), + true + ) :: + StructField( + "struct", + StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", DecimalType(20, 0), true) :: Nil + ), + true + ) :: + StructField( + "structWithArrayFields", + StructType( + StructField("field1", ArrayType(LongType, true), true) :: + StructField("field2", ArrayType(StringType, true), true) :: Nil + ), + true + ) :: Nil + ) + assert(expectedSchema === xmlDF.schema) + } + + test("complex arrays") { + val xmlDF = readData(complexFieldAndType2) + val expectedSchemaArrayOfArray1 = new StructType().add( + "arrayOfArray1", + ArrayType( + new StructType() + .add("array", ArrayType(new StructType().add("item", ArrayType(LongType)))) + ) + ) + assert(xmlDF.select("arrayOfArray1").schema === expectedSchemaArrayOfArray1) + checkAnswer( + xmlDF.select("arrayOfArray1"), + Row( + Seq( + Row(Seq(Row(Seq(5)))), + Row(Seq(Row(Seq(6, 7)), Row(Seq(8)))) + ) + ) :: Nil + ) + val expectedSchemaArrayOfArray2 = new StructType().add( + "arrayOfArray2", + ArrayType( + new StructType() + .add( + "array", + ArrayType( + new StructType().add( + "item", + ArrayType( + new StructType() + .add("inner1", StringType) + .add("inner2", ArrayType(StringType)) + .add("inner3", ArrayType(new StructType().add("inner4", ArrayType(LongType)))) + ) + ) + ) + ) + ) + ) + assert(xmlDF.select("arrayOfArray2").schema === expectedSchemaArrayOfArray2) + checkAnswer( + xmlDF.select("arrayOfArray2"), + Row( + Seq( + Row(Seq(Row(Seq(Row("str1", null, null))))), + Row( + Seq( + Row(null), + Row(Seq(Row(null, Seq("str3", "str33"), null), Row("str11", Seq("str4"), null))) + ) + ), + Row(Seq(Row(Seq(Row(null, null, Seq(Row(Seq(2, 3)), Row(null))))))) + ) + ) :: Nil + ) + } + + test("Complex field and type inferring with null in sampling") { + val xmlDF = readData(xmlNullStruct) + val expectedSchema = StructType( + StructField( + "headers", + StructType( + StructField("Charset", StringType, true) :: + StructField("Host", StringType, true) :: Nil + ), + true + ) :: + StructField("ip", StringType, true) :: + StructField("nullstr", StringType, true) :: Nil + ) + + assert(expectedSchema === xmlDF.schema) + checkAnswer( + xmlDF.select("nullStr", "headers.Host"), + Seq(Row("", "1.abc.com"), Row("", null), Row("", null), Row("", null)) + ) + } + + test("empty records") { + val emptyDF = readData(emptyRecords) + val expectedSchema = new StructType() + .add( + "a", + new StructType() + .add( + "struct", + StructType(StructField("b", StructType(StructField("c", StringType) :: Nil)) :: Nil))) + .add( + "b", + new StructType() + .add( + "item", + ArrayType( + new StructType().add("c", StructType(StructField("struct", StringType) :: Nil))))) + assert(emptyDF.schema === expectedSchema) + } + +} From fe5998538596fa6ec3fd79c426a5664322d0c0ed Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 12 Mar 2024 10:40:19 +0800 Subject: [PATCH 36/50] [MINOR][DOCS] Remove the extra text on page sql-error-conditions-sqlstates ### What changes were proposed in this pull request? The pr aims to remove the extra text `.. include:: /shared/replacements.md` on page `sql-error-conditions-sqlstates.md`. ### Why are the changes needed? - Eliminate misunderstandings https://spark.apache.org/docs/latest/sql-error-conditions-sqlstates.html image - The file `shared/replacements.md` does not exist in the `spark codebase`. ### Does this PR introduce _any_ user-facing change? Yes, only for web page. ### How was this patch tested? Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45469 from panbingkun/minor_fix_sql-error-conditions-sqlstates. Authored-by: panbingkun Signed-off-by: Kent Yao --- docs/sql-error-conditions-sqlstates.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/sql-error-conditions-sqlstates.md b/docs/sql-error-conditions-sqlstates.md index b142c7340537f..06bd5bb861ca4 100644 --- a/docs/sql-error-conditions-sqlstates.md +++ b/docs/sql-error-conditions-sqlstates.md @@ -739,6 +739,3 @@ Spark SQL uses the following `SQLSTATE` classes: - - -.. include:: /shared/replacements.md From 09bdf2a9334f210ece4c23d0f3324f81113330d0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 12 Mar 2024 10:57:06 +0800 Subject: [PATCH 37/50] [SPARK-46043][SQL][FOLLOWUP] do not resolve v2 table provider with custom session catalog ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/43949 to fix a breaking change. Spark allows people to provide a custom session catalog, which may return custom v2 tables based on the table provider. #43949 resolves the table provider earlier than the custom session catalog, and may break custom session catalogs. This PR fixes it by not resolving table provider if custom session catalog is present. ### Why are the changes needed? avoid breaking custom session catalogs ### Does this PR introduce _any_ user-facing change? no, #43949 is not released yet. ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #45440 from cloud-fan/fix. Authored-by: Wenchen Fan Signed-off-by: Kent Yao --- .../sql/execution/datasources/v2/V2SessionCatalog.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 15166c8229034..3d6de985a62f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -81,10 +81,17 @@ class V2SessionCatalog(catalog: SessionCatalog) new CaseInsensitiveStringMap(propertiesWithPath.asJava) } + private def hasCustomSessionCatalog: Boolean = { + catalog.conf.contains(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key) + } + override def loadTable(ident: Identifier): Table = { try { val table = catalog.getTableMetadata(ident.asTableIdentifier) - if (table.provider.isDefined) { + // The custom session catalog may extend `DelegatingCatalogExtension` and rely on the returned + // table here. To avoid breaking it we do not resolve the table provider and still return + // `V1Table` if the custom session catalog is present. + if (table.provider.isDefined && !hasCustomSessionCatalog) { val qualifiedTableName = QualifiedTableName(table.database, table.identifier.table) // Check if the table is in the v1 table cache to skip the v2 table lookup. if (catalog.getCachedTable(qualifiedTableName) != null) { From f53dc08230b7a758227f02fd75d2b446721c139f Mon Sep 17 00:00:00 2001 From: Chaoqin Li Date: Tue, 12 Mar 2024 12:35:18 +0900 Subject: [PATCH 38/50] [SPARK-46962][SS][PYTHON] Add interface for python streaming data source API and implement python worker to run python streaming data source ### What changes were proposed in this pull request? This is the first PR the implement the support to implement streaming data source through python API. Implement python worker to run python streaming data source and communicate with JVM through socket. Create a PythonMicrobatchStream to invoke RPC function call. This happens in the spark driver. For each python streaming data source instance there will be a long live python worker process created. Inside the python process, the python streaming reader will receive function call and parameter from JVM PythonMicroBatchStream and send back result through socket. ### Why are the changes needed? In preparation for support of development of streaming data source in Python. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. PythonMicroBatchStream plan offset and partitions by invoking function call through socket correctly and handle error correctly. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45023 from chaoqin-li1123/python_table. Lead-authored-by: Chaoqin Li Co-authored-by: Hyukjin Kwon Co-authored-by: chaoqin-li1123 <55518381+chaoqin-li1123@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- .../main/resources/error/error-classes.json | 6 + docs/sql-error-conditions.md | 6 + python/pyspark/errors/error_classes.py | 5 + python/pyspark/sql/datasource.py | 153 ++++++++++++ .../python_streaming_source_runner.py | 167 +++++++++++++ .../sql/errors/QueryExecutionErrors.scala | 9 + .../v2/python/PythonMicroBatchStream.scala | 68 +++++ .../datasources/v2/python/PythonScan.scala | 32 ++- .../python/UserDefinedPythonDataSource.scala | 2 +- .../python/PythonStreamingSourceRunner.scala | 202 +++++++++++++++ .../python/PythonDataSourceSuite.scala | 20 +- .../PythonStreamingDataSourceSuite.scala | 233 ++++++++++++++++++ 12 files changed, 883 insertions(+), 20 deletions(-) create mode 100644 python/pyspark/sql/streaming/python_streaming_source_runner.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index afe81b8e9bea7..3d130fdce301d 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3197,6 +3197,12 @@ ], "sqlState" : "38000" }, + "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR" : { + "message" : [ + "Failed when Python streaming data source perform : " + ], + "sqlState" : "38000" + }, "RECURSIVE_PROTOBUF_SCHEMA" : { "message" : [ "Found recursive reference in Protobuf schema, which can not be processed by Spark by default: . try setting the option `recursive.fields.max.depth` 0 to 10. Going beyond 10 levels of recursion is not allowed." diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 0695ed28b7fc8..2cddb6a94c14a 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1931,6 +1931,12 @@ Protobuf type not yet supported: ``. Failed to `` Python data source ``: `` +### PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR + +[SQLSTATE: 38000](sql-error-conditions-sqlstates.html#class-38-external-routine-exception) + +Failed when Python streaming data source perform ``: `` + ### RECURSIVE_PROTOBUF_SCHEMA [SQLSTATE: 42K0G](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index c9a7cfbf356ea..1e21ad3543e92 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -812,6 +812,11 @@ "Randomness of hash of string should be disabled via PYTHONHASHSEED." ] }, + "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR": { + "message": [ + "Failed when running Python streaming data source: " + ] + }, "PYTHON_VERSION_MISMATCH": { "message": [ "Python in worker has different version: than that in driver: , PySpark cannot run with different minor versions.", diff --git a/python/pyspark/sql/datasource.py b/python/pyspark/sql/datasource.py index 1e50f82702431..b98729f04332d 100644 --- a/python/pyspark/sql/datasource.py +++ b/python/pyspark/sql/datasource.py @@ -29,6 +29,7 @@ __all__ = [ "DataSource", "DataSourceReader", + "DataSourceStreamReader", "DataSourceWriter", "DataSourceRegistration", "InputPartition", @@ -159,6 +160,27 @@ def writer(self, schema: StructType, overwrite: bool) -> "DataSourceWriter": message_parameters={"feature": "writer"}, ) + def streamReader(self, schema: StructType) -> "DataSourceStreamReader": + """ + Returns a ``DataSourceStreamReader`` instance for reading streaming data. + + The implementation is required for readable streaming data sources. + + Parameters + ---------- + schema : StructType + The schema of the data to be read. + + Returns + ------- + reader : DataSourceStreamReader + A reader instance for this streaming data source. + """ + raise PySparkNotImplementedError( + error_class="NOT_IMPLEMENTED", + message_parameters={"feature": "streamReader"}, + ) + class InputPartition: """ @@ -298,6 +320,137 @@ def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]: ... +class DataSourceStreamReader(ABC): + """ + A base class for streaming data source readers. Data source stream readers are responsible + for outputting data from a streaming data source. + + .. versionadded: 4.0.0 + """ + + def initialOffset(self) -> dict: + """ + Return the initial offset of the streaming data source. + A new streaming query starts reading data from the initial offset. + If Spark is restarting an existing query, it will restart from the check-pointed offset + rather than the initial one. + + Returns + ------- + dict + A dict or recursive dict whose key and value are primitive types, which includes + Integer, String and Boolean. + + Examples + -------- + >>> def initialOffset(self): + ... return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}} + """ + + ... + raise PySparkNotImplementedError( + error_class="NOT_IMPLEMENTED", + message_parameters={"feature": "initialOffset"}, + ) + + def latestOffset(self) -> dict: + """ + Returns the most recent offset available. + + Returns + ------- + dict + A dict or recursive dict whose key and value are primitive types, which includes + Integer, String and Boolean. + + Examples + -------- + >>> def latestOffset(self): + ... return {"parititon-1": {"index": 3, "closed": True}, "partition-2": {"index": 5}} + """ + ... + raise PySparkNotImplementedError( + error_class="NOT_IMPLEMENTED", + message_parameters={"feature": "latestOffset"}, + ) + + def partitions(self, start: dict, end: dict) -> Sequence[InputPartition]: + """ + Returns a list of InputPartition given the start and end offsets. Each InputPartition + represents a data split that can be processed by one Spark task. + + Parameters + ---------- + start : dict + The start offset of the microbatch to plan partitioning. + end : dict + The end offset of the microbatch to plan partitioning. + + Returns + ------- + Sequence[InputPartition] + A sequence of partitions for this data source. Each partition value + must be an instance of `InputPartition` or a subclass of it. + """ + ... + raise PySparkNotImplementedError( + error_class="NOT_IMPLEMENTED", + message_parameters={"feature": "partitions"}, + ) + + @abstractmethod + def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]: + """ + Generates data for a given partition and returns an iterator of tuples or rows. + + This method is invoked once per partition to read the data. Implementing + this method is required for stream reader. You can initialize any + non-serializable resources required for reading data from the data source + within this method. + + Notes + ----- + This method is static and stateless. You shouldn't access mutable class member + or keep in memory state between different invocations of read(). + + Parameters + ---------- + partition : InputPartition + The partition to read. It must be one of the partition values returned by + ``partitions()``. + + Returns + ------- + Iterator[Tuple] or Iterator[Row] + An iterator of tuples or rows. Each tuple or row will be converted to a row + in the final DataFrame. + """ + ... + raise PySparkNotImplementedError( + error_class="NOT_IMPLEMENTED", + message_parameters={"feature": "read"}, + ) + + def commit(self, end: dict) -> None: + """ + Informs the source that Spark has completed processing all data for offsets less than or + equal to `end` and will only request offsets greater than `end` in the future. + + Parameters + ---------- + end : dict + The latest offset that the streaming query has processed for this source. + """ + ... + + def stop(self) -> None: + """ + Stop this source and free any resources it has allocated. + Invoked when the streaming query terminated. + """ + ... + + class DataSourceWriter(ABC): """ A base class for data source writers. Data source writers are responsible for saving diff --git a/python/pyspark/sql/streaming/python_streaming_source_runner.py b/python/pyspark/sql/streaming/python_streaming_source_runner.py new file mode 100644 index 0000000000000..8dbac431a8ba6 --- /dev/null +++ b/python/pyspark/sql/streaming/python_streaming_source_runner.py @@ -0,0 +1,167 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import sys +import json +from typing import IO + +from pyspark.accumulators import _accumulatorRegistry +from pyspark.errors import IllegalArgumentException, PySparkAssertionError, PySparkRuntimeError +from pyspark.java_gateway import local_connect_and_auth +from pyspark.serializers import ( + read_int, + write_int, + write_with_length, + SpecialLengths, +) +from pyspark.sql.datasource import DataSource, DataSourceStreamReader +from pyspark.sql.types import ( + _parse_datatype_json_string, + StructType, +) +from pyspark.util import handle_worker_exception +from pyspark.worker_util import ( + check_python_version, + read_command, + pickleSer, + send_accumulator_updates, + setup_memory_limits, + setup_spark_files, + utf8_deserializer, +) + +INITIAL_OFFSET_FUNC_ID = 884 +LATEST_OFFSET_FUNC_ID = 885 +PARTITIONS_FUNC_ID = 886 +COMMIT_FUNC_ID = 887 + + +def initial_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None: + offset = reader.initialOffset() + write_with_length(json.dumps(offset).encode("utf-8"), outfile) + + +def latest_offset_func(reader: DataSourceStreamReader, outfile: IO) -> None: + offset = reader.latestOffset() + write_with_length(json.dumps(offset).encode("utf-8"), outfile) + + +def partitions_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None: + start_offset = json.loads(utf8_deserializer.loads(infile)) + end_offset = json.loads(utf8_deserializer.loads(infile)) + partitions = reader.partitions(start_offset, end_offset) + # Return the serialized partition values. + write_int(len(partitions), outfile) + for partition in partitions: + pickleSer._write_with_length(partition, outfile) + + +def commit_func(reader: DataSourceStreamReader, infile: IO, outfile: IO) -> None: + end_offset = json.loads(utf8_deserializer.loads(infile)) + reader.commit(end_offset) + write_int(0, outfile) + + +def main(infile: IO, outfile: IO) -> None: + try: + check_python_version(infile) + setup_spark_files(infile) + + memory_limit_mb = int(os.environ.get("PYSPARK_PLANNER_MEMORY_MB", "-1")) + setup_memory_limits(memory_limit_mb) + + _accumulatorRegistry.clear() + + # Receive the data source instance. + data_source = read_command(pickleSer, infile) + + if not isinstance(data_source, DataSource): + raise PySparkAssertionError( + error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH", + message_parameters={ + "expected": "a Python data source instance of type 'DataSource'", + "actual": f"'{type(data_source).__name__}'", + }, + ) + + # Receive the data source output schema. + schema_json = utf8_deserializer.loads(infile) + schema = _parse_datatype_json_string(schema_json) + if not isinstance(schema, StructType): + raise PySparkAssertionError( + error_class="PYTHON_DATA_SOURCE_TYPE_MISMATCH", + message_parameters={ + "expected": "an output schema of type 'StructType'", + "actual": f"'{type(schema).__name__}'", + }, + ) + + # Instantiate data source reader. + try: + reader = data_source.streamReader(schema=schema) + # Initialization succeed. + write_int(0, outfile) + outfile.flush() + + # handle method call from socket + while True: + func_id = read_int(infile) + if func_id == INITIAL_OFFSET_FUNC_ID: + initial_offset_func(reader, outfile) + elif func_id == LATEST_OFFSET_FUNC_ID: + latest_offset_func(reader, outfile) + elif func_id == PARTITIONS_FUNC_ID: + partitions_func(reader, infile, outfile) + elif func_id == COMMIT_FUNC_ID: + commit_func(reader, infile, outfile) + else: + raise IllegalArgumentException( + error_class="UNSUPPORTED_OPERATION", + message_parameters={ + "operation": "Function call id not recognized by stream reader" + }, + ) + outfile.flush() + except Exception as e: + error_msg = "data source {} throw exception: {}".format(data_source.name, e) + raise PySparkRuntimeError( + error_class="PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + message_parameters={"error": error_msg}, + ) + finally: + reader.stop() + except BaseException as e: + handle_worker_exception(e, outfile) + sys.exit(-1) + send_accumulator_updates(outfile) + + # check end of stream + if read_int(infile) == SpecialLengths.END_OF_STREAM: + write_int(SpecialLengths.END_OF_STREAM, outfile) + else: + # write a different value to tell JVM to not reuse this worker + write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) + sys.exit(-1) + + +if __name__ == "__main__": + # Read information about how to connect back to the JVM from the environment. + java_port = int(os.environ["PYTHON_WORKER_FACTORY_PORT"]) + auth_secret = os.environ["PYTHON_WORKER_FACTORY_SECRET"] + (sock_file, _) = local_connect_and_auth(java_port, auth_secret) + main(sock_file, sock_file) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 8280826fafcb1..32482015b3eda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -1758,6 +1758,15 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "outputMode" -> outputMode.toString())) } + def pythonStreamingDataSourceRuntimeError( + action: String, + message: String): SparkException = { + new SparkException( + errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + messageParameters = Map("action" -> action, "msg" -> message), + cause = null) + } + def invalidCatalogNameError(name: String): Throwable = { new SparkException( errorClass = "_LEGACY_ERROR_TEMP_2212", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala new file mode 100644 index 0000000000000..4e77f33c24f09 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonMicroBatchStream.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.python + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.execution.python.PythonStreamingSourceRunner +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class PythonStreamingSourceOffset(json: String) extends Offset + +case class PythonStreamingSourcePartition(partition: Array[Byte]) extends InputPartition + +class PythonMicroBatchStream( + ds: PythonDataSourceV2, + shortName: String, + outputSchema: StructType, + options: CaseInsensitiveStringMap + ) extends MicroBatchStream with Logging { + private def createDataSourceFunc = + ds.source.createPythonFunction( + ds.getOrCreateDataSourceInPython(shortName, options, Some(outputSchema)).dataSource) + + private val runner: PythonStreamingSourceRunner = + new PythonStreamingSourceRunner(createDataSourceFunc, outputSchema) + runner.init() + + override def initialOffset(): Offset = PythonStreamingSourceOffset(runner.initialOffset()) + + override def latestOffset(): Offset = PythonStreamingSourceOffset(runner.latestOffset()) + + override def planInputPartitions(start: Offset, end: Offset): Array[InputPartition] = { + runner.partitions(start.asInstanceOf[PythonStreamingSourceOffset].json, + end.asInstanceOf[PythonStreamingSourceOffset].json).map(PythonStreamingSourcePartition(_)) + } + + override def createReaderFactory(): PartitionReaderFactory = { + // TODO(SPARK-47107): fill in the implementation. + null + } + + override def commit(end: Offset): Unit = { + runner.commit(end.asInstanceOf[PythonStreamingSourceOffset].json) + } + + override def stop(): Unit = { + runner.stop() + } + + override def deserializeOffset(json: String): Offset = PythonStreamingSourceOffset(json) +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala index 75cbe38b13979..bcddf66fc1615 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/PythonScan.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2.python import org.apache.spark.JobArtifactSet import org.apache.spark.sql.connector.metric.CustomMetric import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -27,9 +28,27 @@ class PythonScan( ds: PythonDataSourceV2, shortName: String, outputSchema: StructType, - options: CaseInsensitiveStringMap) extends Batch with Scan { + options: CaseInsensitiveStringMap) extends Scan { - private[this] val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) + override def toBatch: Batch = new PythonBatch(ds, shortName, outputSchema, options) + + override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = + new PythonMicroBatchStream(ds, shortName, outputSchema, options) + + override def description: String = "(Python)" + + override def readSchema(): StructType = outputSchema + + override def supportedCustomMetrics(): Array[CustomMetric] = + ds.source.createPythonMetrics() +} + +class PythonBatch( + ds: PythonDataSourceV2, + shortName: String, + outputSchema: StructType, + options: CaseInsensitiveStringMap) extends Batch { + private val jobArtifactUUID = JobArtifactSet.getCurrentJobArtifactState.map(_.uuid) private lazy val infoInPython: PythonDataSourceReadInfo = { ds.source.createReadInfoInPython( @@ -45,13 +64,4 @@ class PythonScan( new PythonPartitionReaderFactory( ds.source, readerFunc, outputSchema, jobArtifactUUID) } - - override def toBatch: Batch = this - - override def description: String = "(Python)" - - override def readSchema(): StructType = outputSchema - - override def supportedCustomMetrics(): Array[CustomMetric] = - ds.source.createPythonMetrics() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala index 0e5f359ee76f4..9b8219c4dc2d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/python/UserDefinedPythonDataSource.scala @@ -150,7 +150,7 @@ case class UserDefinedPythonDataSource(dataSourceCls: PythonFunction) { taskMetrics.map { case (k, v) => new PythonCustomTaskMetric(k, v)}.toArray } - private def createPythonFunction(pickledFunc: Array[Byte]): PythonFunction = { + def createPythonFunction(pickledFunc: Array[Byte]): PythonFunction = { SimplePythonFunction( command = pickledFunc.toImmutableArraySeq, envVars = dataSourceCls.envVars, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala new file mode 100644 index 0000000000000..4862f8220f06c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonStreamingSourceRunner.scala @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.sql.execution.python + +import java.io.{BufferedInputStream, BufferedOutputStream, DataInputStream, DataOutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ + +import org.apache.spark.SparkEnv +import org.apache.spark.api.python.{PythonFunction, PythonWorker, PythonWorkerFactory, PythonWorkerUtils, SpecialLengths} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.BUFFER_SIZE +import org.apache.spark.internal.config.Python.PYTHON_AUTH_SOCKET_TIMEOUT +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.types.StructType + +object PythonStreamingSourceRunner { + // When the python process for python_streaming_source_runner receives one of the + // integers below, it will invoke the corresponding function of StreamReader instance. + val INITIAL_OFFSET_FUNC_ID = 884 + val LATEST_OFFSET_FUNC_ID = 885 + val PARTITIONS_FUNC_ID = 886 + val COMMIT_FUNC_ID = 887 +} + +/** + * This class is a proxy to invoke methods in Python DataSourceStreamReader from JVM. + * A runner spawns a python worker process. In the main function, set up communication + * between JVM and python process through socket and create a DataSourceStreamReader instance. + * In an infinite loop, the python worker process poll information(function name and parameters) + * from the socket, invoke the corresponding method of StreamReader and send return value to JVM. + */ +class PythonStreamingSourceRunner( + func: PythonFunction, + outputSchema: StructType) extends Logging { + val workerModule = "pyspark.sql.streaming.python_streaming_source_runner" + + private val conf = SparkEnv.get.conf + private val bufferSize: Int = conf.get(BUFFER_SIZE) + private val authSocketTimeout = conf.get(PYTHON_AUTH_SOCKET_TIMEOUT) + + private val envVars: java.util.Map[String, String] = func.envVars + private val pythonExec: String = func.pythonExec + private var pythonWorker: Option[PythonWorker] = None + private var pythonWorkerFactory: Option[PythonWorkerFactory] = None + private val pythonVer: String = func.pythonVer + + private var dataOut: DataOutputStream = null + private var dataIn: DataInputStream = null + + import PythonStreamingSourceRunner._ + + /** + * Initializes the Python worker for running the streaming source. + */ + def init(): Unit = { + logInfo(s"Initializing Python runner pythonExec: $pythonExec") + val env = SparkEnv.get + + val localdir = env.blockManager.diskBlockManager.localDirs.map(f => f.getPath()).mkString(",") + envVars.put("SPARK_LOCAL_DIRS", localdir) + + envVars.put("SPARK_AUTH_SOCKET_TIMEOUT", authSocketTimeout.toString) + envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString) + + val workerFactory = + new PythonWorkerFactory(pythonExec, workerModule, envVars.asScala.toMap) + val (worker: PythonWorker, _) = workerFactory.createSimpleWorker(blockingMode = true) + pythonWorker = Some(worker) + pythonWorkerFactory = Some(workerFactory) + + val stream = new BufferedOutputStream( + pythonWorker.get.channel.socket().getOutputStream, bufferSize) + dataOut = new DataOutputStream(stream) + + PythonWorkerUtils.writePythonVersion(pythonVer, dataOut) + + val pythonIncludes = func.pythonIncludes.asScala.toSet + PythonWorkerUtils.writeSparkFiles(Some("streaming_job"), pythonIncludes, dataOut) + + // Send the user function to python process + PythonWorkerUtils.writePythonFunction(func, dataOut) + + // Send output schema + PythonWorkerUtils.writeUTF(outputSchema.json, dataOut) + + dataOut.flush() + + dataIn = new DataInputStream( + new BufferedInputStream(pythonWorker.get.channel.socket().getInputStream, bufferSize)) + + val initStatus = dataIn.readInt() + if (initStatus == SpecialLengths.PYTHON_EXCEPTION_THROWN) { + val msg = PythonWorkerUtils.readUTF(dataIn) + throw QueryCompilationErrors.pythonDataSourceError( + action = "plan", tpe = "initialize source", msg = msg) + } + } + + /** + * Invokes latestOffset() function of the stream reader and receive the return value. + */ + def latestOffset(): String = { + dataOut.writeInt(LATEST_OFFSET_FUNC_ID) + dataOut.flush() + val len = dataIn.readInt() + if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) { + val msg = PythonWorkerUtils.readUTF(dataIn) + throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError( + action = "latestOffset", msg) + } + PythonWorkerUtils.readUTF(len, dataIn) + } + + /** + * Invokes initialOffset() function of the stream reader and receive the return value. + */ + def initialOffset(): String = { + dataOut.writeInt(INITIAL_OFFSET_FUNC_ID) + dataOut.flush() + val len = dataIn.readInt() + if (len == SpecialLengths.PYTHON_EXCEPTION_THROWN) { + val msg = PythonWorkerUtils.readUTF(dataIn) + throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError( + action = "initialOffset", msg) + } + PythonWorkerUtils.readUTF(len, dataIn) + } + + /** + * Invokes partitions(start, end) function of the stream reader and receive the return value. + */ + def partitions(start: String, end: String): Array[Array[Byte]] = { + dataOut.writeInt(PARTITIONS_FUNC_ID) + PythonWorkerUtils.writeUTF(start, dataOut) + PythonWorkerUtils.writeUTF(end, dataOut) + dataOut.flush() + // Receive the list of partitions, if any. + val pickledPartitions = ArrayBuffer.empty[Array[Byte]] + val numPartitions = dataIn.readInt() + if (numPartitions == SpecialLengths.PYTHON_EXCEPTION_THROWN) { + val msg = PythonWorkerUtils.readUTF(dataIn) + throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError( + action = "planPartitions", msg) + } + for (_ <- 0 until numPartitions) { + val pickledPartition: Array[Byte] = PythonWorkerUtils.readBytes(dataIn) + pickledPartitions.append(pickledPartition) + } + pickledPartitions.toArray + } + + /** + * Invokes commit(end) function of the stream reader and receive the return value. + */ + def commit(end: String): Unit = { + dataOut.writeInt(COMMIT_FUNC_ID) + PythonWorkerUtils.writeUTF(end, dataOut) + dataOut.flush() + val status = dataIn.readInt() + if (status == SpecialLengths.PYTHON_EXCEPTION_THROWN) { + val msg = PythonWorkerUtils.readUTF(dataIn) + throw QueryExecutionErrors.pythonStreamingDataSourceRuntimeError( + action = "commitSource", msg) + } + } + + /** + * Stop the python worker process and invoke stop() on stream reader. + */ + def stop(): Unit = { + logInfo(s"Stopping streaming runner for module: $workerModule.") + try { + pythonWorkerFactory.foreach { factory => + pythonWorker.foreach { worker => + factory.stopWorker(worker) + factory.stop() + } + } + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala index c2f5864b7345f..b207afeae1068 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala @@ -28,13 +28,9 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -class PythonDataSourceSuite extends QueryTest with SharedSparkSession { - import IntegratedUDFTestUtils._ - - setupTestData() +abstract class PythonDataSourceSuiteBase extends QueryTest with SharedSparkSession { - private def dataSourceName = "SimpleDataSource" - private val simpleDataSourceReaderScript: String = + protected val simpleDataSourceReaderScript: String = """ |from pyspark.sql.datasource import DataSourceReader, InputPartition |class SimpleDataSourceReader(DataSourceReader): @@ -45,8 +41,8 @@ class PythonDataSourceSuite extends QueryTest with SharedSparkSession { | yield (1, partition.value) | yield (2, partition.value) |""".stripMargin - private val staticSourceName = "custom_source" - private var tempDir: File = _ + protected val staticSourceName = "custom_source" + protected var tempDir: File = _ override def beforeAll(): Unit = { // Create a Python Data Source package before starting up the Spark Session @@ -90,6 +86,14 @@ class PythonDataSourceSuite extends QueryTest with SharedSparkSession { } } + setupTestData() + + protected def dataSourceName = "SimpleDataSource" +} + +class PythonDataSourceSuite extends PythonDataSourceSuiteBase { + import IntegratedUDFTestUtils._ + test("SPARK-45917: automatic registration of Python Data Source") { assume(shouldTestPandasUDFs) val df = spark.read.format(staticSourceName).load() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala new file mode 100644 index 0000000000000..f022e353edd71 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.spark.SparkException +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.IntegratedUDFTestUtils.{createUserDefinedPythonDataSource, shouldTestPandasUDFs} +import org.apache.spark.sql.execution.datasources.v2.python.{PythonDataSourceV2, PythonMicroBatchStream, PythonStreamingSourceOffset} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase { + + protected def simpleDataStreamReaderScript: String = + """ + |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition + | + |class SimpleDataStreamReader(DataSourceStreamReader): + | def initialOffset(self): + | return {"offset": {"partition-1": 0}} + | def latestOffset(self): + | return {"offset": {"partition-1": 2}} + | def partitions(self, start: dict, end: dict): + | start_index = start["offset"]["partition-1"] + | end_index = end["offset"]["partition-1"] + | return [InputPartition(i) for i in range(start_index, end_index)] + | def commit(self, end: dict): + | 1 + 2 + | def read(self, partition): + | yield (0, partition.value) + | yield (1, partition.value) + | yield (2, partition.value) + |""".stripMargin + + protected def errorDataStreamReaderScript: String = + """ + |from pyspark.sql.datasource import DataSourceStreamReader, InputPartition + | + |class ErrorDataStreamReader(DataSourceStreamReader): + | def initialOffset(self): + | raise Exception("error reading initial offset") + | def latestOffset(self): + | raise Exception("error reading latest offset") + | def partitions(self, start: dict, end: dict): + | raise Exception("error planning partitions") + | def commit(self, end: dict): + | raise Exception("error committing offset") + | def read(self, partition): + | yield (0, partition.value) + | yield (1, partition.value) + | yield (2, partition.value) + |""".stripMargin + + private val errorDataSourceName = "ErrorDataSource" + + test("simple data stream source") { + assume(shouldTestPandasUDFs) + val dataSourceScript = + s""" + |from pyspark.sql.datasource import DataSource + |$simpleDataStreamReaderScript + | + |class $dataSourceName(DataSource): + | def streamReader(self, schema): + | return SimpleDataStreamReader() + |""".stripMargin + val inputSchema = StructType.fromDDL("input BINARY") + + val dataSource = createUserDefinedPythonDataSource(dataSourceName, dataSourceScript) + spark.dataSource.registerPython(dataSourceName, dataSource) + val pythonDs = new PythonDataSourceV2 + pythonDs.setShortName("SimpleDataSource") + val stream = new PythonMicroBatchStream( + pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty()) + + val initialOffset = stream.initialOffset() + assert(initialOffset.json == "{\"offset\": {\"partition-1\": 0}}") + for (_ <- 1 to 50) { + val offset = stream.latestOffset() + assert(offset.json == "{\"offset\": {\"partition-1\": 2}}") + assert(stream.planInputPartitions(initialOffset, offset).size == 2) + stream.commit(offset) + } + stream.stop() + } + + test("Error creating stream reader") { + assume(shouldTestPandasUDFs) + val dataSourceScript = + s""" + |from pyspark.sql.datasource import DataSource + |class $dataSourceName(DataSource): + | def streamReader(self, schema): + | raise Exception("error creating stream reader") + |""".stripMargin + val dataSource = createUserDefinedPythonDataSource( + name = dataSourceName, pythonScript = dataSourceScript) + spark.dataSource.registerPython(dataSourceName, dataSource) + val pythonDs = new PythonDataSourceV2 + pythonDs.setShortName("SimpleDataSource") + val inputSchema = StructType.fromDDL("input BINARY") + val err = intercept[AnalysisException] { + new PythonMicroBatchStream( + pythonDs, dataSourceName, inputSchema, CaseInsensitiveStringMap.empty()) + } + assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getMessage.contains("error creating stream reader")) + } + + test("Method not implemented in stream reader") { + assume(shouldTestPandasUDFs) + val dataSourceScript = + s""" + |from pyspark.sql.datasource import DataSource + |from pyspark.sql.datasource import DataSourceStreamReader + |class ErrorDataStreamReader(DataSourceStreamReader): + | def read(self, partition): + | yield (0, partition.value) + | + |class $errorDataSourceName(DataSource): + | def streamReader(self, schema): + | return ErrorDataStreamReader() + |""".stripMargin + val inputSchema = StructType.fromDDL("input BINARY") + + val dataSource = createUserDefinedPythonDataSource(errorDataSourceName, dataSourceScript) + spark.dataSource.registerPython(errorDataSourceName, dataSource) + val pythonDs = new PythonDataSourceV2 + pythonDs.setShortName("ErrorDataSource") + + def testMicroBatchStreamError(action: String, msg: String) + (func: PythonMicroBatchStream => Unit): Unit = { + val stream = new PythonMicroBatchStream( + pythonDs, errorDataSourceName, inputSchema, CaseInsensitiveStringMap.empty()) + val err = intercept[SparkException] { + func(stream) + } + checkErrorMatchPVals(err, + errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + parameters = Map( + "action" -> action, + "msg" -> "(.|\\n)*" + )) + assert(err.getMessage.contains(msg)) + assert(err.getMessage.contains("ErrorDataSource")) + stream.stop() + } + + testMicroBatchStreamError( + "initialOffset", "[NOT_IMPLEMENTED] initialOffset is not implemented") { + stream => stream.initialOffset() + } + + testMicroBatchStreamError( + "latestOffset", "[NOT_IMPLEMENTED] latestOffset is not implemented") { + stream => stream.latestOffset() + } + + val offset = PythonStreamingSourceOffset("{\"offset\": \"2\"}") + testMicroBatchStreamError( + "planPartitions", "[NOT_IMPLEMENTED] partitions is not implemented") { + stream => stream.planInputPartitions(offset, offset) + } + } + + test("Error in stream reader") { + assume(shouldTestPandasUDFs) + val dataSourceScript = + s""" + |from pyspark.sql.datasource import DataSource + |$errorDataStreamReaderScript + | + |class $errorDataSourceName(DataSource): + | def streamReader(self, schema): + | return ErrorDataStreamReader() + |""".stripMargin + val inputSchema = StructType.fromDDL("input BINARY") + + val dataSource = createUserDefinedPythonDataSource(errorDataSourceName, dataSourceScript) + spark.dataSource.registerPython(errorDataSourceName, dataSource) + val pythonDs = new PythonDataSourceV2 + pythonDs.setShortName("ErrorDataSource") + val offset = PythonStreamingSourceOffset("{\"offset\": \"2\"}") + + def testMicroBatchStreamError(action: String, msg: String) + (func: PythonMicroBatchStream => Unit): Unit = { + val stream = new PythonMicroBatchStream( + pythonDs, errorDataSourceName, inputSchema, CaseInsensitiveStringMap.empty()) + val err = intercept[SparkException] { + func(stream) + } + checkErrorMatchPVals(err, + errorClass = "PYTHON_STREAMING_DATA_SOURCE_RUNTIME_ERROR", + parameters = Map( + "action" -> action, + "msg" -> "(.|\\n)*" + )) + assert(err.getMessage.contains(msg)) + assert(err.getMessage.contains("ErrorDataSource")) + stream.stop() + } + + testMicroBatchStreamError("initialOffset", "error reading initial offset") { + stream => stream.initialOffset() + } + + testMicroBatchStreamError("latestOffset", "error reading latest offset") { + stream => stream.latestOffset() + } + + testMicroBatchStreamError("planPartitions", "error planning partitions") { + stream => stream.planInputPartitions(offset, offset) + } + + testMicroBatchStreamError("commitSource", "error committing offset") { + stream => stream.commit(offset) + } + } +} From e778ce689dcbe5e75ce5781a03cf9d8466910cd2 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 12 Mar 2024 14:27:50 +0900 Subject: [PATCH 39/50] [SPARK-47250][SS] Add additional validations and NERF changes for RocksDB state provider and use of column families ### What changes were proposed in this pull request? Add additional validations and NERF changes for RocksDB state provider and use of col families ### Why are the changes needed? Improve error handling and migrating errors to NERF. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added new unit tests StateStoreSuite ``` ===== POSSIBLE THREAD LEAK IN SUITE o.a.s.sql.execution.streaming.state.StateStoreSuite, threads: shuffle-boss-36-1 (daemon=true), ForkJoinPool.commonPool-worker-1 (daemon=true), ForkJoinPool.commonPool-worker-3 (daemon=true), rpc-boss-33-1 (daemon=true), ForkJoinPool.commonPool-worker-2 (daemon=true) ===== [info] Run completed in 2 minutes, 57 seconds. [info] Total number of tests run: 151 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 151, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` RocksDBSuite ``` [info] Run completed in 4 minutes, 54 seconds. [info] Total number of tests run: 188 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 188, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #45360 from anishshri-db/task/SPARK-47250. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim --- .../main/resources/error/error-classes.json | 10 +- docs/sql-error-conditions.md | 10 +- .../state/HDFSBackedStateStoreProvider.scala | 33 +++-- .../execution/streaming/state/RocksDB.scala | 92 +++++++++---- .../state/RocksDBStateStoreProvider.scala | 4 +- .../streaming/state/StateStoreErrors.scala | 49 ++++--- .../streaming/state/RocksDBSuite.scala | 124 ++++++++++++++++-- .../streaming/state/StateStoreSuite.scala | 63 +++++++++ .../streaming/state/ValueStateSuite.scala | 2 +- 9 files changed, 318 insertions(+), 69 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 3d130fdce301d..99fbc585f9813 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -3371,9 +3371,9 @@ ], "sqlState" : "0A000" }, - "STATE_STORE_CANNOT_REMOVE_DEFAULT_COLUMN_FAMILY" : { + "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME" : { "message" : [ - "Failed to remove default column family with reserved name=." + "Failed to perform column family operation= with invalid name=. Column family name cannot be empty or include leading/trailing spaces or use the reserved keyword=default" ], "sqlState" : "42802" }, @@ -3396,6 +3396,12 @@ ], "sqlState" : "XXKST" }, + "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY" : { + "message" : [ + "State store operation= not supported on missing column family=." + ], + "sqlState" : "42802" + }, "STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST" : { "message" : [ "Static partition column is also specified in the column list." diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index 2cddb6a94c14a..b6b159f277c02 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -2105,11 +2105,11 @@ The SQL config `` cannot be found. Please verify that the config exists Star (*) is not allowed in a select list when GROUP BY an ordinal position is used. -### STATE_STORE_CANNOT_REMOVE_DEFAULT_COLUMN_FAMILY +### STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME [SQLSTATE: 42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) -Failed to remove default column family with reserved name=``. +Failed to perform column family operation=`` with invalid name=``. Column family name cannot be empty or include leading/trailing spaces or use the reserved keyword=default ### STATE_STORE_HANDLE_NOT_INITIALIZED @@ -2130,6 +2130,12 @@ Store does not support multiple values per key `` operation not supported with `` +### STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY + +[SQLSTATE: 42802](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +State store operation=`` not supported on missing column family=``. + ### STATIC_PARTITION_COLUMN_IN_INSERT_COLUMN_LIST [SQLSTATE: 42713](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 01e2e7f26083c..edb95615d5887 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -73,6 +73,8 @@ import org.apache.spark.util.ArrayImplicits._ */ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with Logging { + private val providerName = "HDFSBackedStateStoreProvider" + class HDFSBackedReadStateStore(val version: Long, map: HDFSBackedStateStoreMap) extends ReadStateStore { @@ -124,14 +126,25 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with numColsPrefixKey: Int, valueSchema: StructType, useMultipleValuesPerKey: Boolean = false): Unit = { - throw StateStoreErrors.multipleColumnFamiliesNotSupported("HDFSStateStoreProvider") + throw StateStoreErrors.multipleColumnFamiliesNotSupported(providerName) + } + + // Multiple col families are not supported with HDFSBackedStateStoreProvider. Throw an exception + // if the user tries to use a non-default col family. + private def assertUseOfDefaultColFamily(colFamilyName: String): Unit = { + if (colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME) { + + throw StateStoreErrors.multipleColumnFamiliesNotSupported(providerName) + } } override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { + assertUseOfDefaultColFamily(colFamilyName) mapToUpdate.get(key) } override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { + assertUseOfDefaultColFamily(colFamilyName) require(value != null, "Cannot put a null value") verify(state == UPDATING, "Cannot put after already committed or aborted") val keyCopy = key.copy() @@ -141,6 +154,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { + assertUseOfDefaultColFamily(colFamilyName) verify(state == UPDATING, "Cannot remove after already committed or aborted") val prevValue = mapToUpdate.remove(key) if (prevValue != null) { @@ -179,10 +193,14 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with * Get an iterator of all the store data. * This can be called only after committing all the updates made in the current thread. */ - override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = mapToUpdate.iterator() + override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { + assertUseOfDefaultColFamily(colFamilyName) + mapToUpdate.iterator() + } override def prefixScan(prefixKey: UnsafeRow, colFamilyName: String): Iterator[UnsafeRowPair] = { + assertUseOfDefaultColFamily(colFamilyName) mapToUpdate.prefixScan(prefixKey) } @@ -211,18 +229,17 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with } override def removeColFamilyIfExists(colFamilyName: String): Unit = { - throw StateStoreErrors.removingColumnFamiliesNotSupported( - "HDFSBackedStateStoreProvider") + throw StateStoreErrors.removingColumnFamiliesNotSupported(providerName) } override def valuesIterator(key: UnsafeRow, colFamilyName: String): Iterator[UnsafeRow] = { - throw StateStoreErrors.unsupportedOperationException("multipleValuesPerKey", "HDFSStateStore") + throw StateStoreErrors.unsupportedOperationException("multipleValuesPerKey", providerName) } override def merge(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { - throw StateStoreErrors.unsupportedOperationException("merge", "HDFSStateStore") + throw StateStoreErrors.unsupportedOperationException("merge", providerName) } } @@ -280,11 +297,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with // TODO: add support for multiple col families with HDFSBackedStateStoreProvider if (useColumnFamilies) { - throw StateStoreErrors.multipleColumnFamiliesNotSupported("HDFSStateStoreProvider") + throw StateStoreErrors.multipleColumnFamiliesNotSupported(providerName) } if (useMultipleValuesPerKey) { - throw StateStoreErrors.unsupportedOperationException("multipleValuesPerKey", "HDFSStateStore") + throw StateStoreErrors.unsupportedOperationException("multipleValuesPerKey", providerName) } require((keySchema.length == 0 && numColsPrefixKey == 0) || diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 41cab78df1950..4437cc5583d46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -34,7 +34,7 @@ import org.rocksdb.{RocksDB => NativeRocksDB, _} import org.rocksdb.CompressionType._ import org.rocksdb.TickerType._ -import org.apache.spark.{SparkUnsupportedOperationException, TaskContext} +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.errors.QueryExecutionErrors @@ -242,26 +242,75 @@ class RocksDB( loadedVersion = endVersion } + /** + * Function to check if the column family exists in the state store instance. + * @param colFamilyName - name of the column family + * @return - true if the column family exists, false otherwise + */ private def checkColFamilyExists(colFamilyName: String): Boolean = { colFamilyNameToHandleMap.contains(colFamilyName) } - private def verifyColFamilyExists(colFamilyName: String): Unit = { - if (useColumnFamilies && !checkColFamilyExists(colFamilyName)) { - throw new RuntimeException(s"Column family with name=$colFamilyName does not exist") + private val multColFamiliesDisabledStr = "multiple column families disabled in " + + "RocksDBStateStoreProvider" + + /** + * Function to verify invariants for column family based operations such as get, put, remove etc. + * @param operationName - name of the store operation + * @param colFamilyName - name of the column family + */ + private def verifyColFamilyOperations( + operationName: String, + colFamilyName: String): Unit = { + if (colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME) { + // if the state store instance does not support multiple column families, throw an exception + if (!useColumnFamilies) { + throw StateStoreErrors.unsupportedOperationException(operationName, + multColFamiliesDisabledStr) + } + + // if the column family name is empty or contains leading/trailing whitespaces, throw an + // exception + if (colFamilyName.isEmpty || colFamilyName.trim != colFamilyName) { + throw StateStoreErrors.cannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) + } + + // if the column family does not exist, throw an exception + if (!checkColFamilyExists(colFamilyName)) { + throw StateStoreErrors.unsupportedOperationOnMissingColumnFamily(operationName, + colFamilyName) + } } } /** - * Create RocksDB column family, if not created already + * Function to verify invariants for column family creation or deletion operations. + * @param operationName - name of the store operation + * @param colFamilyName - name of the column family */ - def createColFamilyIfAbsent(colFamilyName: String): Unit = { - if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) { - throw new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_3197", - messageParameters = Map("colFamilyName" -> colFamilyName).toMap) + private def verifyColFamilyCreationOrDeletion( + operationName: String, + colFamilyName: String): Unit = { + // if the state store instance does not support multiple column families, throw an exception + if (!useColumnFamilies) { + throw StateStoreErrors.unsupportedOperationException(operationName, + multColFamiliesDisabledStr) + } + + // if the column family name is empty or contains leading/trailing whitespaces + // or using the reserved "default" column family, throw an exception + if (colFamilyName.isEmpty + || colFamilyName.trim != colFamilyName + || colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) { + throw StateStoreErrors.cannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) } + } + /** + * Create RocksDB column family, if not created already + */ + def createColFamilyIfAbsent(colFamilyName: String): Unit = { + verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName) if (!checkColFamilyExists(colFamilyName)) { assert(db != null) val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions) @@ -274,10 +323,7 @@ class RocksDB( * Remove RocksDB column family, if exists */ def removeColFamilyIfExists(colFamilyName: String): Unit = { - if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) { - throw StateStoreErrors.cannotRemoveDefaultColumnFamily(colFamilyName) - } - + verifyColFamilyCreationOrDeletion("remove_col_family", colFamilyName) if (checkColFamilyExists(colFamilyName)) { assert(db != null) val handle = colFamilyNameToHandleMap(colFamilyName) @@ -293,7 +339,7 @@ class RocksDB( def get( key: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("get", colFamilyName) db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) } @@ -305,7 +351,7 @@ class RocksDB( key: Array[Byte], value: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("put", colFamilyName) if (conf.trackTotalNumberOfRows) { val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) if (oldValue == null) { @@ -337,10 +383,10 @@ class RocksDB( value: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { if (!useColumnFamilies) { - throw new RuntimeException("Merge operation uses changelog checkpointing v2 which" + - " requires column families to be enabled.") + throw StateStoreErrors.unsupportedOperationException("merge", + multColFamiliesDisabledStr) } - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("merge", colFamilyName) if (conf.trackTotalNumberOfRows) { val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) @@ -360,7 +406,7 @@ class RocksDB( def remove( key: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("remove", colFamilyName) if (conf.trackTotalNumberOfRows) { val value = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) if (value != null) { @@ -380,7 +426,7 @@ class RocksDB( */ def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("iterator", colFamilyName) val iter = db.newIterator(colFamilyNameToHandleMap(colFamilyName)) logInfo(s"Getting iterator from version $loadedVersion") @@ -409,7 +455,7 @@ class RocksDB( } private def countKeys(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Long = { - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("countKeys", colFamilyName) val iter = db.newIterator(colFamilyNameToHandleMap(colFamilyName)) try { @@ -431,7 +477,7 @@ class RocksDB( def prefixScan(prefix: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { - verifyColFamilyExists(colFamilyName) + verifyColFamilyOperations("prefixScan", colFamilyName) val iter = db.newIterator(colFamilyNameToHandleMap(colFamilyName)) iter.seek(prefix) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 7374abdbde98f..721d8aa030798 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -101,14 +101,14 @@ private[sql] class RocksDBStateStoreProvider override def merge(key: UnsafeRow, value: UnsafeRow, colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - verify(state == UPDATING, "Cannot put after already committed or aborted") + verify(state == UPDATING, "Cannot merge after already committed or aborted") val kvEncoder = keyValueEncoderMap.get(colFamilyName) val keyEncoder = kvEncoder._1 val valueEncoder = kvEncoder._2 verify(valueEncoder.supportsMultipleValuesPerKey, "Merge operation requires an encoder" + " which supports multiple values for a single key") verify(key != null, "Key cannot be null") - require(value != null, "Cannot put a null value") + require(value != null, "Cannot merge a null value") rocksDB.merge(keyEncoder.encodeKey(key), valueEncoder.encodeValue(value), colFamilyName) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala index 6f4c3d4c9675a..8a0276557f8f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala @@ -32,25 +32,30 @@ object StateStoreErrors { ) } + def unsupportedOperationOnMissingColumnFamily(operationName: String, colFamilyName: String): + StateStoreUnsupportedOperationOnMissingColumnFamily = { + new StateStoreUnsupportedOperationOnMissingColumnFamily(operationName, colFamilyName) + } + def multipleColumnFamiliesNotSupported(stateStoreProvider: String): StateStoreMultipleColumnFamiliesNotSupportedException = { - new StateStoreMultipleColumnFamiliesNotSupportedException(stateStoreProvider) - } + new StateStoreMultipleColumnFamiliesNotSupportedException(stateStoreProvider) + } def removingColumnFamiliesNotSupported(stateStoreProvider: String): StateStoreRemovingColumnFamiliesNotSupportedException = { - new StateStoreRemovingColumnFamiliesNotSupportedException(stateStoreProvider) - } + new StateStoreRemovingColumnFamiliesNotSupportedException(stateStoreProvider) + } - def cannotRemoveDefaultColumnFamily(colFamilyName: String): - StateStoreCannotRemoveDefaultColumnFamily = { - new StateStoreCannotRemoveDefaultColumnFamily(colFamilyName) - } + def cannotUseColumnFamilyWithInvalidName(operationName: String, colFamilyName: String): + StateStoreCannotUseColumnFamilyWithInvalidName = { + new StateStoreCannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) + } def unsupportedOperationException(operationName: String, entity: String): StateStoreUnsupportedOperationException = { - new StateStoreUnsupportedOperationException(operationName, entity) - } + new StateStoreUnsupportedOperationException(operationName, entity) + } def requireNonNullStateValue(value: Any, stateName: String): Unit = { SparkException.require(value != null, @@ -68,23 +73,25 @@ object StateStoreErrors { class StateStoreMultipleColumnFamiliesNotSupportedException(stateStoreProvider: String) extends SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", - messageParameters = Map("stateStoreProvider" -> stateStoreProvider) - ) + messageParameters = Map("stateStoreProvider" -> stateStoreProvider)) + class StateStoreRemovingColumnFamiliesNotSupportedException(stateStoreProvider: String) extends SparkUnsupportedOperationException( errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_REMOVING_COLUMN_FAMILIES", - messageParameters = Map("stateStoreProvider" -> stateStoreProvider) - ) + messageParameters = Map("stateStoreProvider" -> stateStoreProvider)) -class StateStoreCannotRemoveDefaultColumnFamily(colFamilyName: String) +class StateStoreCannotUseColumnFamilyWithInvalidName(operationName: String, colFamilyName: String) extends SparkUnsupportedOperationException( - errorClass = "STATE_STORE_CANNOT_REMOVE_DEFAULT_COLUMN_FAMILY", - messageParameters = Map("colFamilyName" -> colFamilyName) - ) - + errorClass = "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME", + messageParameters = Map("operationName" -> operationName, "colFamilyName" -> colFamilyName)) class StateStoreUnsupportedOperationException(operationType: String, entity: String) extends SparkUnsupportedOperationException( errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", - messageParameters = Map("operationType" -> operationType, "entity" -> entity) - ) + messageParameters = Map("operationType" -> operationType, "entity" -> entity)) + +class StateStoreUnsupportedOperationOnMissingColumnFamily( + operationType: String, + colFamilyName: String) extends SparkUnsupportedOperationException( + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + messageParameters = Map("operationType" -> operationType, "colFamilyName" -> colFamilyName)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 0bc3828318da0..a7d4ab3623407 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -536,6 +536,110 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } + testWithColumnFamilies(s"RocksDB: column family creation with invalid names", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + + val conf = RocksDBConf().copy() + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => + Seq("default", "", " ", " ", " default", " default ").foreach { colFamilyName => + val ex = intercept[SparkUnsupportedOperationException] { + db.createColFamilyIfAbsent(colFamilyName) + } + + if (!colFamiliesEnabled) { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + parameters = Map( + "operationType" -> "create_col_family", + "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + ), + matchPVals = true + ) + } else { + checkError( + ex, + errorClass = "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME", + parameters = Map( + "operationName" -> "create_col_family", + "colFamilyName" -> colFamilyName + ), + matchPVals = true + ) + } + } + } + } + + private def verifyStoreOperationUnsupported( + operationName: String, + colFamiliesEnabled: Boolean, + colFamilyName: String) + (testFn: => Unit): Unit = { + val ex = intercept[SparkUnsupportedOperationException] { + testFn + } + + if (!colFamiliesEnabled) { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + parameters = Map( + "operationType" -> operationName, + "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + ), + matchPVals = true + ) + } else { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + parameters = Map( + "operationType" -> operationName, + "colFamilyName" -> colFamilyName + ), + matchPVals = true + ) + } + } + + testWithColumnFamilies(s"RocksDB: operations on absent column family", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + val remoteDir = Utils.createTempDir().toString + new File(remoteDir).delete() // to make sure that the directory gets created + + val conf = RocksDBConf().copy() + withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => + db.load(0) + val colFamilyName = "test" + verifyStoreOperationUnsupported("put", colFamiliesEnabled, colFamilyName) { + db.put("a", "1", colFamilyName) + } + + verifyStoreOperationUnsupported("remove", colFamiliesEnabled, colFamilyName) { + db.remove("a", colFamilyName) + } + + verifyStoreOperationUnsupported("get", colFamiliesEnabled, colFamilyName) { + db.get("a", colFamilyName) + } + + verifyStoreOperationUnsupported("iterator", colFamiliesEnabled, colFamilyName) { + db.iterator(colFamilyName) + } + + verifyStoreOperationUnsupported("merge", colFamiliesEnabled, colFamilyName) { + db.merge("a", "1", colFamilyName) + } + + verifyStoreOperationUnsupported("prefixScan", colFamiliesEnabled, colFamilyName) { + db.prefixScan("a", colFamilyName) + } + } + } + testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load " + s"with multiple column families", TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => @@ -545,13 +649,6 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared val colFamily2: String = "xyz" val conf = RocksDBConf().copy() - withDB(remoteDir, conf = conf, useColumnFamilies = true) { db => - val ex = intercept[Exception] { - db.createColFamilyIfAbsent("default") - } - ex.getCause.isInstanceOf[UnsupportedOperationException] - } - withDB(remoteDir, conf = conf, useColumnFamilies = true) { db => db.createColFamilyIfAbsent(colFamily1) db.createColFamilyIfAbsent(colFamily2) @@ -572,7 +669,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } withDB(remoteDir, conf = conf, version = 0, useColumnFamilies = true) { db => - val ex = intercept[Exception] { + val ex = intercept[SparkUnsupportedOperationException] { // version 0 can be loaded again assert(toStr(db.get("a", colFamily1)) === null) assert(iterator(db, colFamily1).isEmpty) @@ -581,8 +678,15 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared assert(toStr(db.get("a", colFamily2)) === null) assert(iterator(db, colFamily2).isEmpty) } - assert(ex.isInstanceOf[RuntimeException]) - assert(ex.getMessage.contains("does not exist")) + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + parameters = Map( + "operationType" -> "get", + "colFamilyName" -> colFamily1 + ), + matchPVals = true + ) } withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = true) { db => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index a8c7fc05f21ec..64b3e75ea9763 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -134,6 +134,69 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } } + private def verifyStoreOperationUnsupported(operationName: String)(testFn: => Unit): Unit = { + if (operationName != "merge") { + val ex = intercept[SparkUnsupportedOperationException] { + testFn + } + checkError( + ex, + errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", + parameters = Map( + "stateStoreProvider" -> "HDFSBackedStateStoreProvider" + ), + matchPVals = true + ) + } else { + val ex = intercept[SparkUnsupportedOperationException] { + testFn + } + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + parameters = Map( + "operationType" -> operationName, + "entity" -> "HDFSBackedStateStoreProvider" + ), + matchPVals = true + ) + + } + } + + test("get, put, remove etc operations on non-default col family should fail") { + tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0, + minDeltasForSnapshot = 5)) { provider => + val store = provider.getStore(0) + val keyRow = dataToKeyRow("a", 0) + val valueRow = dataToValueRow(1) + val colFamilyName = "test" + verifyStoreOperationUnsupported("put") { + store.put(keyRow, valueRow, colFamilyName) + } + + verifyStoreOperationUnsupported("remove") { + store.remove(keyRow, colFamilyName) + } + + verifyStoreOperationUnsupported("get") { + store.get(keyRow, colFamilyName) + } + + verifyStoreOperationUnsupported("merge") { + store.merge(keyRow, valueRow, colFamilyName) + } + + verifyStoreOperationUnsupported("iterator") { + store.iterator(colFamilyName) + } + + verifyStoreOperationUnsupported("prefixScan") { + store.prefixScan(keyRow, colFamilyName) + } + } + } + test("failure after committing with MAX_BATCHES_TO_RETAIN_IN_MEMORY set to 1") { tryWithProviderResource(newStoreProvider(opId = Random.nextInt(), partition = 0, numOfVersToRetainInMemory = 1)) { provider => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala index 71462cb4b6435..40e31239895cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala @@ -217,7 +217,7 @@ class ValueStateSuite extends SharedSparkSession ex, errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES", parameters = Map( - "stateStoreProvider" -> "HDFSStateStoreProvider" + "stateStoreProvider" -> "HDFSBackedStateStoreProvider" ), matchPVals = true ) From e1865811cac9c809c89b4b1512843e4b716c3e46 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 12 Mar 2024 15:52:43 +0800 Subject: [PATCH 40/50] [MINOR] Minor English fixes ### What changes were proposed in this pull request? Minor English grammar and wording fixes. ### Why are the changes needed? They're not strictly needed, but give the project a tiny bit more polish. ### Does this PR introduce _any_ user-facing change? Yes, some user-facing errors have been tweaked. ### How was this patch tested? No testing beyond CI. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #45461 from nchammas/minor-wording-tweaks. Authored-by: Nicholas Chammas Signed-off-by: Ruifeng Zheng --- common/utils/src/main/resources/error/error-classes.json | 8 ++++---- .../apache/spark/storage/BlockManagerMasterEndpoint.scala | 6 +++--- docs/sql-error-conditions.md | 2 +- .../apache/spark/sql/execution/ui/UISeleniumSuite.scala | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 99fbc585f9813..93846e66df521 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -40,7 +40,7 @@ "AMBIGUOUS_COLUMN_REFERENCE" : { "message" : [ "Column is ambiguous. It's because you joined several DataFrame together, and some of these DataFrames are the same.", - "This column points to one of the DataFrame but Spark is unable to figure out which one.", + "This column points to one of the DataFrames but Spark is unable to figure out which one.", "Please alias the DataFrames with different names via `DataFrame.alias` before joining them,", "and specify the column using qualified name, e.g. `df.alias(\"a\").join(df.alias(\"b\"), col(\"a.id\") > col(\"b.id\"))`." ], @@ -6184,17 +6184,17 @@ }, "_LEGACY_ERROR_TEMP_2109" : { "message" : [ - "Cannot build HashedRelation with more than 1/3 billions unique keys." + "Cannot build HashedRelation with more than 1/3 billion unique keys." ] }, "_LEGACY_ERROR_TEMP_2110" : { "message" : [ - "Can not build a HashedRelation that is larger than 8G." + "Cannot build a HashedRelation that is larger than 8G." ] }, "_LEGACY_ERROR_TEMP_2111" : { "message" : [ - "failed to push a row into ." + "Failed to push a row into ." ] }, "_LEGACY_ERROR_TEMP_2112" : { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index b4920c7cb841d..5dd536eeb3046 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -323,11 +323,11 @@ class BlockManagerMasterEndpoint( val isAlive = try { driverEndpoint.askSync[Boolean](CoarseGrainedClusterMessages.IsExecutorAlive(executorId)) } catch { - // ignore the non-fatal error from driverEndpoint since the caller doesn't really - // care about the return result of removing blocks. And so we could avoid breaking + // Ignore the non-fatal error from driverEndpoint since the caller doesn't really + // care about the return result of removing blocks. That way we avoid breaking // down the whole application. case NonFatal(e) => - logError(s"Fail to know the executor $executorId is alive or not.", e) + logError(s"Cannot determine whether executor $executorId is alive or not.", e) false } if (!isAlive) { diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index b6b159f277c02..165f0a1d94af1 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -71,7 +71,7 @@ Column or field `` is ambiguous and has `` matches. [SQLSTATE: 42702](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) Column `` is ambiguous. It's because you joined several DataFrame together, and some of these DataFrames are the same. -This column points to one of the DataFrame but Spark is unable to figure out which one. +This column points to one of the DataFrames but Spark is unable to figure out which one. Please alias the DataFrames with different names via `DataFrame.alias` before joining them, and specify the column using qualified name, e.g. `df.alias("a").join(df.alias("b"), col("a.id") > col("b.id"))`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala index dc617046c4303..111e233c04e32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala @@ -102,7 +102,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser { test("SPARK-44801: Analyzer failure shall show the query in failed table") { spark = creatSparkSessionWithUI - intercept[Exception](spark.sql("SELECT * FROM I_AM_A_INVISIBLE_TABLE").isEmpty) + intercept[Exception](spark.sql("SELECT * FROM I_AM_AN_INVISIBLE_TABLE").isEmpty) eventually(timeout(10.seconds), interval(100.milliseconds)) { val sd = findErrorMessageOnSQLUI() assert(sd.size === 1, "Analyze fail shall show the query in failed table") From 95aec974d8e7b2e648d2ad20d01ed585de80c498 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 12 Mar 2024 17:22:26 +0900 Subject: [PATCH 41/50] [SPARK-47347][PYTHON][CONNECT][TESTS] Factor session-related tests out of `test_connect_basic` ### What changes were proposed in this pull request? Factor session-related tests out of `test_connect_basic` ### Why are the changes needed? for testing parallelism ### 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 #45472 from zhengruifeng/add_test_connect_session. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- dev/sparktestsupport/modules.py | 1 + .../sql/tests/connect/test_connect_basic.py | 439 ---------------- .../sql/tests/connect/test_connect_session.py | 488 ++++++++++++++++++ 3 files changed, 489 insertions(+), 439 deletions(-) create mode 100644 python/pyspark/sql/tests/connect/test_connect_session.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index b2c64a8242de2..3ecfeb5f69aac 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -995,6 +995,7 @@ def __hash__(self): "pyspark.sql.tests.connect.test_connect_basic", "pyspark.sql.tests.connect.test_connect_function", "pyspark.sql.tests.connect.test_connect_column", + "pyspark.sql.tests.connect.test_connect_session", "pyspark.sql.tests.connect.test_parity_arrow", "pyspark.sql.tests.connect.test_parity_arrow_python_udf", "pyspark.sql.tests.connect.test_parity_datasources", diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index cb7e286c5371c..b55875b8577da 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -23,19 +23,14 @@ import shutil import string import tempfile -import uuid -from collections import defaultdict from pyspark.errors import ( PySparkAttributeError, PySparkTypeError, - PySparkException, PySparkValueError, - RetriesExceeded, ) from pyspark.errors.exceptions.base import SessionNotSameException from pyspark.sql import SparkSession as PySparkSession, Row -from pyspark.sql.connect.client.retries import RetryPolicy from pyspark.sql.types import ( StructType, StructField, @@ -57,30 +52,25 @@ from pyspark.testing.connectutils import ( should_test_connect, ReusedConnectTestCase, - connect_requirement_message, ) from pyspark.testing.pandasutils import PandasOnSparkTestUtils from pyspark.errors.exceptions.connect import ( AnalysisException, ParseException, SparkConnectException, - SparkUpgradeException, ) if should_test_connect: - import grpc import pandas as pd import numpy as np from pyspark.sql.connect.proto import Expression as ProtoExpression from pyspark.sql.connect.session import SparkSession as RemoteSparkSession - from pyspark.sql.connect.client import DefaultChannelBuilder, ChannelBuilder from pyspark.sql.connect.column import Column from pyspark.sql.connect.readwriter import DataFrameWriterV2 from pyspark.sql.dataframe import DataFrame from pyspark.sql.connect.dataframe import DataFrame as CDataFrame from pyspark.sql import functions as SF from pyspark.sql.connect import functions as CF - from pyspark.sql.connect.client.core import Retrying, SparkConnectClient class SparkConnectSQLTestCase(ReusedConnectTestCase, SQLTestUtils, PandasOnSparkTestUtils): @@ -3410,435 +3400,6 @@ def test_df_caache(self): self.assertTrue(df.is_cached) -class SparkConnectSessionTests(ReusedConnectTestCase): - def setUp(self) -> None: - self.spark = ( - PySparkSession.builder.config(conf=self.conf()) - .appName(self.__class__.__name__) - .remote("local[4]") - .getOrCreate() - ) - - def tearDown(self): - self.spark.stop() - - def _check_no_active_session_error(self, e: PySparkException): - self.check_error(exception=e, error_class="NO_ACTIVE_SESSION", message_parameters=dict()) - - def test_stop_session(self): - df = self.spark.sql("select 1 as a, 2 as b") - catalog = self.spark.catalog - self.spark.stop() - - # _execute_and_fetch - with self.assertRaises(SparkConnectException) as e: - self.spark.sql("select 1") - self._check_no_active_session_error(e.exception) - - with self.assertRaises(SparkConnectException) as e: - catalog.tableExists("table") - self._check_no_active_session_error(e.exception) - - # _execute - with self.assertRaises(SparkConnectException) as e: - self.spark.udf.register("test_func", lambda x: x + 1) - self._check_no_active_session_error(e.exception) - - # _analyze - with self.assertRaises(SparkConnectException) as e: - df._explain_string(extended=True) - self._check_no_active_session_error(e.exception) - - # Config - with self.assertRaises(SparkConnectException) as e: - self.spark.conf.get("some.conf") - self._check_no_active_session_error(e.exception) - - def test_error_enrichment_message(self): - with self.sql_conf( - { - "spark.sql.connect.enrichError.enabled": True, - "spark.sql.connect.serverStacktrace.enabled": False, - "spark.sql.pyspark.jvmStacktrace.enabled": False, - } - ): - name = "test" * 10000 - with self.assertRaises(AnalysisException) as e: - self.spark.sql("select " + name).collect() - self.assertTrue(name in e.exception._message) - self.assertFalse("JVM stacktrace" in e.exception._message) - - def test_error_enrichment_jvm_stacktrace(self): - with self.sql_conf( - { - "spark.sql.connect.enrichError.enabled": True, - "spark.sql.pyspark.jvmStacktrace.enabled": False, - } - ): - with self.sql_conf({"spark.sql.connect.serverStacktrace.enabled": False}): - with self.assertRaises(SparkUpgradeException) as e: - self.spark.sql( - """select from_json( - '{"d": "02-29"}', 'd date', map('dateFormat', 'MM-dd'))""" - ).collect() - self.assertFalse("JVM stacktrace" in e.exception._message) - - with self.sql_conf({"spark.sql.connect.serverStacktrace.enabled": True}): - with self.assertRaises(SparkUpgradeException) as e: - self.spark.sql( - """select from_json( - '{"d": "02-29"}', 'd date', map('dateFormat', 'MM-dd'))""" - ).collect() - self.assertTrue("JVM stacktrace" in str(e.exception)) - self.assertTrue("org.apache.spark.SparkUpgradeException" in str(e.exception)) - self.assertTrue( - "at org.apache.spark.sql.errors.ExecutionErrors" - ".failToParseDateTimeInNewParserError" in str(e.exception) - ) - self.assertTrue("Caused by: java.time.DateTimeException:" in str(e.exception)) - - def test_not_hitting_netty_header_limit(self): - with self.sql_conf({"spark.sql.pyspark.jvmStacktrace.enabled": True}): - with self.assertRaises(AnalysisException): - self.spark.sql("select " + "test" * 1).collect() - - def test_error_stack_trace(self): - with self.sql_conf({"spark.sql.connect.enrichError.enabled": False}): - with self.sql_conf({"spark.sql.pyspark.jvmStacktrace.enabled": True}): - with self.assertRaises(AnalysisException) as e: - self.spark.sql("select x").collect() - self.assertTrue("JVM stacktrace" in str(e.exception)) - self.assertIsNotNone(e.exception.getStackTrace()) - self.assertTrue( - "at org.apache.spark.sql.catalyst.analysis.CheckAnalysis" in str(e.exception) - ) - - with self.sql_conf({"spark.sql.pyspark.jvmStacktrace.enabled": False}): - with self.assertRaises(AnalysisException) as e: - self.spark.sql("select x").collect() - self.assertFalse("JVM stacktrace" in str(e.exception)) - self.assertIsNone(e.exception.getStackTrace()) - self.assertFalse( - "at org.apache.spark.sql.catalyst.analysis.CheckAnalysis" in str(e.exception) - ) - - # Create a new session with a different stack trace size. - self.spark.stop() - spark = ( - PySparkSession.builder.config(conf=self.conf()) - .config("spark.connect.grpc.maxMetadataSize", 128) - .remote("local[4]") - .getOrCreate() - ) - spark.conf.set("spark.sql.connect.enrichError.enabled", False) - spark.conf.set("spark.sql.pyspark.jvmStacktrace.enabled", True) - with self.assertRaises(AnalysisException) as e: - spark.sql("select x").collect() - self.assertTrue("JVM stacktrace" in str(e.exception)) - self.assertIsNotNone(e.exception.getStackTrace()) - self.assertFalse( - "at org.apache.spark.sql.catalyst.analysis.CheckAnalysis" in str(e.exception) - ) - spark.stop() - - def test_can_create_multiple_sessions_to_different_remotes(self): - self.spark.stop() - self.assertIsNotNone(self.spark._client) - # Creates a new remote session. - other = PySparkSession.builder.remote("sc://other.remote:114/").create() - self.assertNotEqual(self.spark, other) - - # Gets currently active session. - same = PySparkSession.builder.remote("sc://other.remote.host:114/").getOrCreate() - self.assertEqual(other, same) - same.release_session_on_close = False # avoid sending release to dummy connection - same.stop() - - # Make sure the environment is clean. - self.spark.stop() - with self.assertRaises(RuntimeError) as e: - PySparkSession.builder.create() - self.assertIn("Create a new SparkSession is only supported with SparkConnect.", str(e)) - - def test_get_message_parameters_without_enriched_error(self): - with self.sql_conf({"spark.sql.connect.enrichError.enabled": False}): - exception = None - try: - self.spark.sql("""SELECT a""") - except AnalysisException as e: - exception = e - - self.assertIsNotNone(exception) - self.assertEqual(exception.getMessageParameters(), {"objectName": "`a`"}) - - def test_custom_channel_builder(self): - # Access self.spark's DefaultChannelBuilder to reuse same endpoint - endpoint = self.spark._client._builder.endpoint - - class CustomChannelBuilder(ChannelBuilder): - def toChannel(self): - return self._insecure_channel(endpoint) - - session = RemoteSparkSession.builder.channelBuilder(CustomChannelBuilder()).create() - session.sql("select 1 + 1") - - -class SparkConnectSessionWithOptionsTest(unittest.TestCase): - def setUp(self) -> None: - self.spark = ( - PySparkSession.builder.config("string", "foo") - .config("integer", 1) - .config("boolean", False) - .appName(self.__class__.__name__) - .remote("local[4]") - .getOrCreate() - ) - - def tearDown(self): - self.spark.stop() - - def test_config(self): - # Config - self.assertEqual(self.spark.conf.get("string"), "foo") - self.assertEqual(self.spark.conf.get("boolean"), "false") - self.assertEqual(self.spark.conf.get("integer"), "1") - - -class TestError(grpc.RpcError, Exception): - def __init__(self, code: grpc.StatusCode): - self._code = code - - def code(self): - return self._code - - -class TestPolicy(RetryPolicy): - # Put a small value for initial backoff so that tests don't spend - # Time waiting - def __init__(self, initial_backoff=10, **kwargs): - super().__init__(initial_backoff=initial_backoff, **kwargs) - - def can_retry(self, exception: BaseException): - return isinstance(exception, TestError) - - -class TestPolicySpecificError(TestPolicy): - def __init__(self, specific_code: grpc.StatusCode, **kwargs): - super().__init__(**kwargs) - self.specific_code = specific_code - - def can_retry(self, exception: BaseException): - return exception.code() == self.specific_code - - -@unittest.skipIf(not should_test_connect, connect_requirement_message) -class RetryTests(unittest.TestCase): - def setUp(self) -> None: - self.call_wrap = defaultdict(int) - - def stub(self, retries, code): - self.call_wrap["attempts"] += 1 - if self.call_wrap["attempts"] < retries: - self.call_wrap["raised"] += 1 - raise TestError(code) - - def test_simple(self): - # Check that max_retries 1 is only one retry so two attempts. - for attempt in Retrying(TestPolicy(max_retries=1)): - with attempt: - self.stub(2, grpc.StatusCode.INTERNAL) - - self.assertEqual(2, self.call_wrap["attempts"]) - self.assertEqual(1, self.call_wrap["raised"]) - - def test_below_limit(self): - # Check that if we have less than 4 retries all is ok. - for attempt in Retrying(TestPolicy(max_retries=4)): - with attempt: - self.stub(2, grpc.StatusCode.INTERNAL) - - self.assertLess(self.call_wrap["attempts"], 4) - self.assertEqual(self.call_wrap["raised"], 1) - - def test_exceed_retries(self): - # Exceed the retries. - with self.assertRaises(RetriesExceeded): - for attempt in Retrying(TestPolicy(max_retries=2)): - with attempt: - self.stub(5, grpc.StatusCode.INTERNAL) - - self.assertLess(self.call_wrap["attempts"], 5) - self.assertEqual(self.call_wrap["raised"], 3) - - def test_throw_not_retriable_error(self): - with self.assertRaises(ValueError): - for attempt in Retrying(TestPolicy(max_retries=2)): - with attempt: - raise ValueError - - def test_specific_exception(self): - # Check that only specific exceptions are retried. - # Check that if we have less than 4 retries all is ok. - policy = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.UNAVAILABLE) - - for attempt in Retrying(policy): - with attempt: - self.stub(2, grpc.StatusCode.UNAVAILABLE) - - self.assertLess(self.call_wrap["attempts"], 4) - self.assertEqual(self.call_wrap["raised"], 1) - - def test_specific_exception_exceed_retries(self): - # Exceed the retries. - policy = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.UNAVAILABLE) - with self.assertRaises(RetriesExceeded): - for attempt in Retrying(policy): - with attempt: - self.stub(5, grpc.StatusCode.UNAVAILABLE) - - self.assertLess(self.call_wrap["attempts"], 4) - self.assertEqual(self.call_wrap["raised"], 3) - - def test_rejected_by_policy(self): - # Test that another error is always thrown. - policy = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.UNAVAILABLE) - - with self.assertRaises(TestError): - for attempt in Retrying(policy): - with attempt: - self.stub(5, grpc.StatusCode.INTERNAL) - - self.assertEqual(self.call_wrap["attempts"], 1) - self.assertEqual(self.call_wrap["raised"], 1) - - def test_multiple_policies(self): - policy1 = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.UNAVAILABLE) - policy2 = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.INTERNAL) - - # Tolerate 2 UNAVAILABLE errors and 4 INTERNAL errors - - error_suply = iter([grpc.StatusCode.UNAVAILABLE] * 2 + [grpc.StatusCode.INTERNAL] * 4) - - for attempt in Retrying([policy1, policy2]): - with attempt: - error = next(error_suply, None) - if error: - raise TestError(error) - - self.assertEqual(next(error_suply, None), None) - - def test_multiple_policies_exceed(self): - policy1 = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.INTERNAL) - policy2 = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.INTERNAL) - - with self.assertRaises(RetriesExceeded): - for attempt in Retrying([policy1, policy2]): - with attempt: - self.stub(10, grpc.StatusCode.INTERNAL) - - self.assertEqual(self.call_wrap["attempts"], 7) - self.assertEqual(self.call_wrap["raised"], 7) - - -@unittest.skipIf(not should_test_connect, connect_requirement_message) -class ChannelBuilderTests(unittest.TestCase): - def test_invalid_connection_strings(self): - invalid = [ - "scc://host:12", - "http://host", - "sc:/host:1234/path", - "sc://host/path", - "sc://host/;parm1;param2", - ] - for i in invalid: - self.assertRaises(PySparkValueError, DefaultChannelBuilder, i) - - def test_sensible_defaults(self): - chan = DefaultChannelBuilder("sc://host") - self.assertFalse(chan.secure, "Default URL is not secure") - - chan = DefaultChannelBuilder("sc://host/;token=abcs") - self.assertTrue(chan.secure, "specifying a token must set the channel to secure") - self.assertRegex( - chan.userAgent, r"^_SPARK_CONNECT_PYTHON spark/[^ ]+ os/[^ ]+ python/[^ ]+$" - ) - chan = DefaultChannelBuilder("sc://host/;use_ssl=abcs") - self.assertFalse(chan.secure, "Garbage in, false out") - - def test_user_agent(self): - chan = DefaultChannelBuilder("sc://host/;user_agent=Agent123%20%2F3.4") - self.assertIn("Agent123 /3.4", chan.userAgent) - - def test_user_agent_len(self): - user_agent = "x" * 2049 - chan = DefaultChannelBuilder(f"sc://host/;user_agent={user_agent}") - with self.assertRaises(SparkConnectException) as err: - chan.userAgent - self.assertRegex(err.exception._message, "'user_agent' parameter should not exceed") - - user_agent = "%C3%A4" * 341 # "%C3%A4" -> "ä"; (341 * 6 = 2046) < 2048 - expected = "ä" * 341 - chan = DefaultChannelBuilder(f"sc://host/;user_agent={user_agent}") - self.assertIn(expected, chan.userAgent) - - def test_valid_channel_creation(self): - chan = DefaultChannelBuilder("sc://host").toChannel() - self.assertIsInstance(chan, grpc.Channel) - - # Sets up a channel without tokens because ssl is not used. - chan = DefaultChannelBuilder("sc://host/;use_ssl=true;token=abc").toChannel() - self.assertIsInstance(chan, grpc.Channel) - - chan = DefaultChannelBuilder("sc://host/;use_ssl=true").toChannel() - self.assertIsInstance(chan, grpc.Channel) - - def test_channel_properties(self): - chan = DefaultChannelBuilder( - "sc://host/;use_ssl=true;token=abc;user_agent=foo;param1=120%2021" - ) - self.assertEqual("host:15002", chan.endpoint) - self.assertIn("foo", chan.userAgent.split(" ")) - self.assertEqual(True, chan.secure) - self.assertEqual("120 21", chan.get("param1")) - - def test_metadata(self): - chan = DefaultChannelBuilder( - "sc://host/;use_ssl=true;token=abc;param1=120%2021;x-my-header=abcd" - ) - md = chan.metadata() - self.assertEqual([("param1", "120 21"), ("x-my-header", "abcd")], md) - - def test_metadata(self): - id = str(uuid.uuid4()) - chan = DefaultChannelBuilder(f"sc://host/;session_id={id}") - self.assertEqual(id, chan.session_id) - - chan = DefaultChannelBuilder( - f"sc://host/;session_id={id};user_agent=acbd;token=abcd;use_ssl=true" - ) - md = chan.metadata() - for kv in md: - self.assertNotIn( - kv[0], - [ - ChannelBuilder.PARAM_SESSION_ID, - ChannelBuilder.PARAM_TOKEN, - ChannelBuilder.PARAM_USER_ID, - ChannelBuilder.PARAM_USER_AGENT, - ChannelBuilder.PARAM_USE_SSL, - ], - "Metadata must not contain fixed params", - ) - - with self.assertRaises(ValueError) as ve: - chan = DefaultChannelBuilder("sc://host/;session_id=abcd") - SparkConnectClient(chan) - self.assertIn("Parameter value session_id must be a valid UUID format", str(ve.exception)) - - chan = DefaultChannelBuilder("sc://host/") - self.assertIsNone(chan.session_id) - - if __name__ == "__main__": from pyspark.sql.tests.connect.test_connect_basic import * # noqa: F401 diff --git a/python/pyspark/sql/tests/connect/test_connect_session.py b/python/pyspark/sql/tests/connect/test_connect_session.py new file mode 100644 index 0000000000000..9dc4d2ee9e497 --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_connect_session.py @@ -0,0 +1,488 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import unittest +import uuid +from collections import defaultdict + + +from pyspark.errors import ( + PySparkException, + PySparkValueError, + RetriesExceeded, +) +from pyspark.sql import SparkSession as PySparkSession +from pyspark.sql.connect.client.retries import RetryPolicy + +from pyspark.testing.connectutils import ( + should_test_connect, + ReusedConnectTestCase, + connect_requirement_message, +) +from pyspark.errors.exceptions.connect import ( + AnalysisException, + SparkConnectException, + SparkUpgradeException, +) + +if should_test_connect: + import grpc + from pyspark.sql.connect.session import SparkSession as RemoteSparkSession + from pyspark.sql.connect.client import DefaultChannelBuilder, ChannelBuilder + from pyspark.sql.connect.client.core import Retrying, SparkConnectClient + + +class SparkConnectSessionTests(ReusedConnectTestCase): + def setUp(self) -> None: + self.spark = ( + PySparkSession.builder.config(conf=self.conf()) + .appName(self.__class__.__name__) + .remote("local[4]") + .getOrCreate() + ) + + def tearDown(self): + self.spark.stop() + + def _check_no_active_session_error(self, e: PySparkException): + self.check_error(exception=e, error_class="NO_ACTIVE_SESSION", message_parameters=dict()) + + def test_stop_session(self): + df = self.spark.sql("select 1 as a, 2 as b") + catalog = self.spark.catalog + self.spark.stop() + + # _execute_and_fetch + with self.assertRaises(SparkConnectException) as e: + self.spark.sql("select 1") + self._check_no_active_session_error(e.exception) + + with self.assertRaises(SparkConnectException) as e: + catalog.tableExists("table") + self._check_no_active_session_error(e.exception) + + # _execute + with self.assertRaises(SparkConnectException) as e: + self.spark.udf.register("test_func", lambda x: x + 1) + self._check_no_active_session_error(e.exception) + + # _analyze + with self.assertRaises(SparkConnectException) as e: + df._explain_string(extended=True) + self._check_no_active_session_error(e.exception) + + # Config + with self.assertRaises(SparkConnectException) as e: + self.spark.conf.get("some.conf") + self._check_no_active_session_error(e.exception) + + def test_error_enrichment_message(self): + with self.sql_conf( + { + "spark.sql.connect.enrichError.enabled": True, + "spark.sql.connect.serverStacktrace.enabled": False, + "spark.sql.pyspark.jvmStacktrace.enabled": False, + } + ): + name = "test" * 10000 + with self.assertRaises(AnalysisException) as e: + self.spark.sql("select " + name).collect() + self.assertTrue(name in e.exception._message) + self.assertFalse("JVM stacktrace" in e.exception._message) + + def test_error_enrichment_jvm_stacktrace(self): + with self.sql_conf( + { + "spark.sql.connect.enrichError.enabled": True, + "spark.sql.pyspark.jvmStacktrace.enabled": False, + } + ): + with self.sql_conf({"spark.sql.connect.serverStacktrace.enabled": False}): + with self.assertRaises(SparkUpgradeException) as e: + self.spark.sql( + """select from_json( + '{"d": "02-29"}', 'd date', map('dateFormat', 'MM-dd'))""" + ).collect() + self.assertFalse("JVM stacktrace" in e.exception._message) + + with self.sql_conf({"spark.sql.connect.serverStacktrace.enabled": True}): + with self.assertRaises(SparkUpgradeException) as e: + self.spark.sql( + """select from_json( + '{"d": "02-29"}', 'd date', map('dateFormat', 'MM-dd'))""" + ).collect() + self.assertTrue("JVM stacktrace" in str(e.exception)) + self.assertTrue("org.apache.spark.SparkUpgradeException" in str(e.exception)) + self.assertTrue( + "at org.apache.spark.sql.errors.ExecutionErrors" + ".failToParseDateTimeInNewParserError" in str(e.exception) + ) + self.assertTrue("Caused by: java.time.DateTimeException:" in str(e.exception)) + + def test_not_hitting_netty_header_limit(self): + with self.sql_conf({"spark.sql.pyspark.jvmStacktrace.enabled": True}): + with self.assertRaises(AnalysisException): + self.spark.sql("select " + "test" * 1).collect() + + def test_error_stack_trace(self): + with self.sql_conf({"spark.sql.connect.enrichError.enabled": False}): + with self.sql_conf({"spark.sql.pyspark.jvmStacktrace.enabled": True}): + with self.assertRaises(AnalysisException) as e: + self.spark.sql("select x").collect() + self.assertTrue("JVM stacktrace" in str(e.exception)) + self.assertIsNotNone(e.exception.getStackTrace()) + self.assertTrue( + "at org.apache.spark.sql.catalyst.analysis.CheckAnalysis" in str(e.exception) + ) + + with self.sql_conf({"spark.sql.pyspark.jvmStacktrace.enabled": False}): + with self.assertRaises(AnalysisException) as e: + self.spark.sql("select x").collect() + self.assertFalse("JVM stacktrace" in str(e.exception)) + self.assertIsNone(e.exception.getStackTrace()) + self.assertFalse( + "at org.apache.spark.sql.catalyst.analysis.CheckAnalysis" in str(e.exception) + ) + + # Create a new session with a different stack trace size. + self.spark.stop() + spark = ( + PySparkSession.builder.config(conf=self.conf()) + .config("spark.connect.grpc.maxMetadataSize", 128) + .remote("local[4]") + .getOrCreate() + ) + spark.conf.set("spark.sql.connect.enrichError.enabled", False) + spark.conf.set("spark.sql.pyspark.jvmStacktrace.enabled", True) + with self.assertRaises(AnalysisException) as e: + spark.sql("select x").collect() + self.assertTrue("JVM stacktrace" in str(e.exception)) + self.assertIsNotNone(e.exception.getStackTrace()) + self.assertFalse( + "at org.apache.spark.sql.catalyst.analysis.CheckAnalysis" in str(e.exception) + ) + spark.stop() + + def test_can_create_multiple_sessions_to_different_remotes(self): + self.spark.stop() + self.assertIsNotNone(self.spark._client) + # Creates a new remote session. + other = PySparkSession.builder.remote("sc://other.remote:114/").create() + self.assertNotEqual(self.spark, other) + + # Gets currently active session. + same = PySparkSession.builder.remote("sc://other.remote.host:114/").getOrCreate() + self.assertEqual(other, same) + same.release_session_on_close = False # avoid sending release to dummy connection + same.stop() + + # Make sure the environment is clean. + self.spark.stop() + with self.assertRaises(RuntimeError) as e: + PySparkSession.builder.create() + self.assertIn("Create a new SparkSession is only supported with SparkConnect.", str(e)) + + def test_get_message_parameters_without_enriched_error(self): + with self.sql_conf({"spark.sql.connect.enrichError.enabled": False}): + exception = None + try: + self.spark.sql("""SELECT a""") + except AnalysisException as e: + exception = e + + self.assertIsNotNone(exception) + self.assertEqual(exception.getMessageParameters(), {"objectName": "`a`"}) + + def test_custom_channel_builder(self): + # Access self.spark's DefaultChannelBuilder to reuse same endpoint + endpoint = self.spark._client._builder.endpoint + + class CustomChannelBuilder(ChannelBuilder): + def toChannel(self): + return self._insecure_channel(endpoint) + + session = RemoteSparkSession.builder.channelBuilder(CustomChannelBuilder()).create() + session.sql("select 1 + 1") + + +class SparkConnectSessionWithOptionsTest(unittest.TestCase): + def setUp(self) -> None: + self.spark = ( + PySparkSession.builder.config("string", "foo") + .config("integer", 1) + .config("boolean", False) + .appName(self.__class__.__name__) + .remote("local[4]") + .getOrCreate() + ) + + def tearDown(self): + self.spark.stop() + + def test_config(self): + # Config + self.assertEqual(self.spark.conf.get("string"), "foo") + self.assertEqual(self.spark.conf.get("boolean"), "false") + self.assertEqual(self.spark.conf.get("integer"), "1") + + +class TestError(grpc.RpcError, Exception): + def __init__(self, code: grpc.StatusCode): + self._code = code + + def code(self): + return self._code + + +class TestPolicy(RetryPolicy): + # Put a small value for initial backoff so that tests don't spend + # Time waiting + def __init__(self, initial_backoff=10, **kwargs): + super().__init__(initial_backoff=initial_backoff, **kwargs) + + def can_retry(self, exception: BaseException): + return isinstance(exception, TestError) + + +class TestPolicySpecificError(TestPolicy): + def __init__(self, specific_code: grpc.StatusCode, **kwargs): + super().__init__(**kwargs) + self.specific_code = specific_code + + def can_retry(self, exception: BaseException): + return exception.code() == self.specific_code + + +@unittest.skipIf(not should_test_connect, connect_requirement_message) +class RetryTests(unittest.TestCase): + def setUp(self) -> None: + self.call_wrap = defaultdict(int) + + def stub(self, retries, code): + self.call_wrap["attempts"] += 1 + if self.call_wrap["attempts"] < retries: + self.call_wrap["raised"] += 1 + raise TestError(code) + + def test_simple(self): + # Check that max_retries 1 is only one retry so two attempts. + for attempt in Retrying(TestPolicy(max_retries=1)): + with attempt: + self.stub(2, grpc.StatusCode.INTERNAL) + + self.assertEqual(2, self.call_wrap["attempts"]) + self.assertEqual(1, self.call_wrap["raised"]) + + def test_below_limit(self): + # Check that if we have less than 4 retries all is ok. + for attempt in Retrying(TestPolicy(max_retries=4)): + with attempt: + self.stub(2, grpc.StatusCode.INTERNAL) + + self.assertLess(self.call_wrap["attempts"], 4) + self.assertEqual(self.call_wrap["raised"], 1) + + def test_exceed_retries(self): + # Exceed the retries. + with self.assertRaises(RetriesExceeded): + for attempt in Retrying(TestPolicy(max_retries=2)): + with attempt: + self.stub(5, grpc.StatusCode.INTERNAL) + + self.assertLess(self.call_wrap["attempts"], 5) + self.assertEqual(self.call_wrap["raised"], 3) + + def test_throw_not_retriable_error(self): + with self.assertRaises(ValueError): + for attempt in Retrying(TestPolicy(max_retries=2)): + with attempt: + raise ValueError + + def test_specific_exception(self): + # Check that only specific exceptions are retried. + # Check that if we have less than 4 retries all is ok. + policy = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.UNAVAILABLE) + + for attempt in Retrying(policy): + with attempt: + self.stub(2, grpc.StatusCode.UNAVAILABLE) + + self.assertLess(self.call_wrap["attempts"], 4) + self.assertEqual(self.call_wrap["raised"], 1) + + def test_specific_exception_exceed_retries(self): + # Exceed the retries. + policy = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.UNAVAILABLE) + with self.assertRaises(RetriesExceeded): + for attempt in Retrying(policy): + with attempt: + self.stub(5, grpc.StatusCode.UNAVAILABLE) + + self.assertLess(self.call_wrap["attempts"], 4) + self.assertEqual(self.call_wrap["raised"], 3) + + def test_rejected_by_policy(self): + # Test that another error is always thrown. + policy = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.UNAVAILABLE) + + with self.assertRaises(TestError): + for attempt in Retrying(policy): + with attempt: + self.stub(5, grpc.StatusCode.INTERNAL) + + self.assertEqual(self.call_wrap["attempts"], 1) + self.assertEqual(self.call_wrap["raised"], 1) + + def test_multiple_policies(self): + policy1 = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.UNAVAILABLE) + policy2 = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.INTERNAL) + + # Tolerate 2 UNAVAILABLE errors and 4 INTERNAL errors + + error_suply = iter([grpc.StatusCode.UNAVAILABLE] * 2 + [grpc.StatusCode.INTERNAL] * 4) + + for attempt in Retrying([policy1, policy2]): + with attempt: + error = next(error_suply, None) + if error: + raise TestError(error) + + self.assertEqual(next(error_suply, None), None) + + def test_multiple_policies_exceed(self): + policy1 = TestPolicySpecificError(max_retries=2, specific_code=grpc.StatusCode.INTERNAL) + policy2 = TestPolicySpecificError(max_retries=4, specific_code=grpc.StatusCode.INTERNAL) + + with self.assertRaises(RetriesExceeded): + for attempt in Retrying([policy1, policy2]): + with attempt: + self.stub(10, grpc.StatusCode.INTERNAL) + + self.assertEqual(self.call_wrap["attempts"], 7) + self.assertEqual(self.call_wrap["raised"], 7) + + +@unittest.skipIf(not should_test_connect, connect_requirement_message) +class ChannelBuilderTests(unittest.TestCase): + def test_invalid_connection_strings(self): + invalid = [ + "scc://host:12", + "http://host", + "sc:/host:1234/path", + "sc://host/path", + "sc://host/;parm1;param2", + ] + for i in invalid: + self.assertRaises(PySparkValueError, DefaultChannelBuilder, i) + + def test_sensible_defaults(self): + chan = DefaultChannelBuilder("sc://host") + self.assertFalse(chan.secure, "Default URL is not secure") + + chan = DefaultChannelBuilder("sc://host/;token=abcs") + self.assertTrue(chan.secure, "specifying a token must set the channel to secure") + self.assertRegex( + chan.userAgent, r"^_SPARK_CONNECT_PYTHON spark/[^ ]+ os/[^ ]+ python/[^ ]+$" + ) + chan = DefaultChannelBuilder("sc://host/;use_ssl=abcs") + self.assertFalse(chan.secure, "Garbage in, false out") + + def test_user_agent(self): + chan = DefaultChannelBuilder("sc://host/;user_agent=Agent123%20%2F3.4") + self.assertIn("Agent123 /3.4", chan.userAgent) + + def test_user_agent_len(self): + user_agent = "x" * 2049 + chan = DefaultChannelBuilder(f"sc://host/;user_agent={user_agent}") + with self.assertRaises(SparkConnectException) as err: + chan.userAgent + self.assertRegex(err.exception._message, "'user_agent' parameter should not exceed") + + user_agent = "%C3%A4" * 341 # "%C3%A4" -> "ä"; (341 * 6 = 2046) < 2048 + expected = "ä" * 341 + chan = DefaultChannelBuilder(f"sc://host/;user_agent={user_agent}") + self.assertIn(expected, chan.userAgent) + + def test_valid_channel_creation(self): + chan = DefaultChannelBuilder("sc://host").toChannel() + self.assertIsInstance(chan, grpc.Channel) + + # Sets up a channel without tokens because ssl is not used. + chan = DefaultChannelBuilder("sc://host/;use_ssl=true;token=abc").toChannel() + self.assertIsInstance(chan, grpc.Channel) + + chan = DefaultChannelBuilder("sc://host/;use_ssl=true").toChannel() + self.assertIsInstance(chan, grpc.Channel) + + def test_channel_properties(self): + chan = DefaultChannelBuilder( + "sc://host/;use_ssl=true;token=abc;user_agent=foo;param1=120%2021" + ) + self.assertEqual("host:15002", chan.endpoint) + self.assertIn("foo", chan.userAgent.split(" ")) + self.assertEqual(True, chan.secure) + self.assertEqual("120 21", chan.get("param1")) + + def test_metadata(self): + chan = DefaultChannelBuilder( + "sc://host/;use_ssl=true;token=abc;param1=120%2021;x-my-header=abcd" + ) + md = chan.metadata() + self.assertEqual([("param1", "120 21"), ("x-my-header", "abcd")], md) + + def test_metadata(self): + id = str(uuid.uuid4()) + chan = DefaultChannelBuilder(f"sc://host/;session_id={id}") + self.assertEqual(id, chan.session_id) + + chan = DefaultChannelBuilder( + f"sc://host/;session_id={id};user_agent=acbd;token=abcd;use_ssl=true" + ) + md = chan.metadata() + for kv in md: + self.assertNotIn( + kv[0], + [ + ChannelBuilder.PARAM_SESSION_ID, + ChannelBuilder.PARAM_TOKEN, + ChannelBuilder.PARAM_USER_ID, + ChannelBuilder.PARAM_USER_AGENT, + ChannelBuilder.PARAM_USE_SSL, + ], + "Metadata must not contain fixed params", + ) + + with self.assertRaises(ValueError) as ve: + chan = DefaultChannelBuilder("sc://host/;session_id=abcd") + SparkConnectClient(chan) + self.assertIn("Parameter value session_id must be a valid UUID format", str(ve.exception)) + + chan = DefaultChannelBuilder("sc://host/") + self.assertIsNone(chan.session_id) + + +if __name__ == "__main__": + from pyspark.sql.tests.connect.test_connect_session import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + + unittest.main(testRunner=testRunner, verbosity=2) From 1f58f4c68e8de03a8b4c314488dd4f342beb8de2 Mon Sep 17 00:00:00 2001 From: Stefan Bukorovic Date: Tue, 12 Mar 2024 14:50:03 +0500 Subject: [PATCH 42/50] [SPARK-47254][SQL] Assign names to the error classes _LEGACY_ERROR_TEMP_325[1-9] ### What changes were proposed in this pull request? In the PR, I propose to assign the proper names to the legacy error classes _LEGACY_ERROR_TEMP_325[1-9], and modify tests in testing suites to reflect these changes and use checkError() function. Also this PR improves the error messages. ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? By running modified test suits: `build/sbt "test:testOnly *DateTimeFormatterHelperSuite"` `build/sbt "test:testOnly *IntervalUtilsSuite"` `build/sbt "test:testOnly *DataTypeSuite"` `build/sbt "test:testOnly *StructTypeSuite"` ### Was this patch authored or co-authored using generative AI tooling? No Closes #45407 from stefanbuk-db/parsing_fields_legacy_errors. Authored-by: Stefan Bukorovic Signed-off-by: Max Gekk --- R/pkg/tests/fulltests/test_streaming.R | 3 +- .../main/resources/error/error-classes.json | 150 ++++++++---- ...tent-behavior-cross-version-error-class.md | 6 + ...ns-invalid-datetime-pattern-error-class.md | 41 ++++ ...ons-invalid-interval-format-error-class.md | 81 +++++++ docs/sql-error-conditions.md | 28 +++ .../spark/ml/tuning/CrossValidatorSuite.scala | 16 +- .../util/DateTimeFormatterHelper.scala | 14 +- .../catalyst/util/SparkIntervalUtils.scala | 113 ++++++--- .../org/apache/spark/sql/types/DataType.scala | 8 +- .../apache/spark/sql/types/StructType.scala | 19 +- .../sql/catalyst/parser/AstBuilder.scala | 3 +- .../catalyst/csv/UnivocityParserSuite.scala | 14 +- .../parser/ExpressionParserSuite.scala | 11 - .../util/DateTimeFormatterHelperSuite.scala | 22 +- .../catalyst/util/IntervalUtilsSuite.scala | 214 ++++++++++++++---- .../spark/sql/types/DataTypeSuite.scala | 8 +- .../spark/sql/types/StructTypeSuite.scala | 12 +- .../analyzer-results/ansi/interval.sql.out | 17 +- .../analyzer-results/interval.sql.out | 17 +- .../sql-tests/results/ansi/interval.sql.out | 17 +- .../datetime-formatting-invalid.sql.out | 46 ++-- .../sql-tests/results/interval.sql.out | 17 +- .../execution/command/DDLParserSuite.scala | 13 -- 24 files changed, 643 insertions(+), 247 deletions(-) create mode 100644 docs/sql-error-conditions-invalid-datetime-pattern-error-class.md create mode 100644 docs/sql-error-conditions-invalid-interval-format-error-class.md diff --git a/R/pkg/tests/fulltests/test_streaming.R b/R/pkg/tests/fulltests/test_streaming.R index 67479726b57c1..88114f8bd82b8 100644 --- a/R/pkg/tests/fulltests/test_streaming.R +++ b/R/pkg/tests/fulltests/test_streaming.R @@ -257,8 +257,7 @@ test_that("Trigger", { "Value for trigger.processingTime must be a non-empty string.") expect_error(write.stream(df, "memory", queryName = "times", outputMode = "append", - trigger.processingTime = "invalid"), - "Error parsing 'invalid' to interval, unrecognized number 'invalid'") + trigger.processingTime = "invalid")) expect_error(write.stream(df, "memory", queryName = "times", outputMode = "append", trigger.once = ""), "Value for trigger.once must be TRUE.") diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 93846e66df521..41fe7c7061655 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1546,6 +1546,13 @@ "2) You can form a valid datetime pattern with the guide from '/sql-ref-datetime-pattern.html'." ] }, + "DATETIME_WEEK_BASED_PATTERN" : { + "message" : [ + "Spark >= 3.0:", + "All week-based patterns are unsupported since Spark 3.0, detected week-based character: .", + "Please use the SQL function EXTRACT instead." + ] + }, "PARSE_DATETIME_BY_NEW_PARSER" : { "message" : [ "Spark >= 3.0:", @@ -1840,6 +1847,24 @@ }, "sqlState" : "HY109" }, + "INVALID_DATETIME_PATTERN" : { + "message" : [ + "Unrecognized datetime pattern: ." + ], + "subClass" : { + "ILLEGAL_CHARACTER" : { + "message" : [ + "Illegal pattern character found in datetime pattern: . Please provide legal character." + ] + }, + "LENGTH" : { + "message" : [ + "Too many letters in datetime pattern: . Please reduce pattern length." + ] + } + }, + "sqlState" : "22007" + }, "INVALID_DEFAULT_VALUE" : { "message" : [ "Failed to execute command because the destination column or variable has a DEFAULT value ," @@ -2105,6 +2130,74 @@ }, "sqlState" : "42000" }, + "INVALID_INTERVAL_FORMAT" : { + "message" : [ + "Error parsing '' to interval. Please ensure that the value provided is in a valid format for defining an interval. You can reference the documentation for the correct format." + ], + "subClass" : { + "ARITHMETIC_EXCEPTION" : { + "message" : [ + "Uncaught arithmetic exception while parsing ''." + ] + }, + "INPUT_IS_EMPTY" : { + "message" : [ + "Interval string cannot be empty." + ] + }, + "INPUT_IS_NULL" : { + "message" : [ + "Interval string cannot be null." + ] + }, + "INVALID_FRACTION" : { + "message" : [ + " cannot have fractional part." + ] + }, + "INVALID_PRECISION" : { + "message" : [ + "Interval can only support nanosecond precision, is out of range." + ] + }, + "INVALID_PREFIX" : { + "message" : [ + "Invalid interval prefix ." + ] + }, + "INVALID_UNIT" : { + "message" : [ + "Invalid unit ." + ] + }, + "INVALID_VALUE" : { + "message" : [ + "Invalid value ." + ] + }, + "MISSING_NUMBER" : { + "message" : [ + "Expect a number after but hit EOL." + ] + }, + "MISSING_UNIT" : { + "message" : [ + "Expect a unit name after but hit EOL." + ] + }, + "UNKNOWN_PARSING_ERROR" : { + "message" : [ + "Unknown error when parsing ." + ] + }, + "UNRECOGNIZED_NUMBER" : { + "message" : [ + "Unrecognized number ." + ] + } + }, + "sqlState" : "22006" + }, "INVALID_INVERSE_DISTRIBUTION_FUNCTION" : { "message" : [ "Invalid inverse distribution function ." @@ -2128,6 +2221,12 @@ }, "sqlState" : "42K0K" }, + "INVALID_JSON_DATA_TYPE" : { + "message" : [ + "Failed to convert the JSON string '' to a data type. Please enter a valid data type." + ], + "sqlState" : "2203G" + }, "INVALID_JSON_ROOT_FIELD" : { "message" : [ "Cannot convert JSON root field to target Spark type." @@ -2852,6 +2951,12 @@ ], "sqlState" : "07501" }, + "NONEXISTENT_FIELD_NAME_IN_LIST" : { + "message" : [ + "Field(s) do(es) not exist. Available fields: " + ], + "sqlState" : "HV091" + }, "NON_FOLDABLE_ARGUMENT" : { "message" : [ "The function requires the parameter to be a foldable expression of the type , but the actual argument is a non-foldable." @@ -7863,51 +7968,6 @@ "Failed to convert the JSON string '' to a field." ] }, - "_LEGACY_ERROR_TEMP_3251" : { - "message" : [ - "Failed to convert the JSON string '' to a data type." - ] - }, - "_LEGACY_ERROR_TEMP_3252" : { - "message" : [ - " does not exist. Available: " - ] - }, - "_LEGACY_ERROR_TEMP_3253" : { - "message" : [ - " do(es) not exist. Available: " - ] - }, - "_LEGACY_ERROR_TEMP_3254" : { - "message" : [ - " does not exist. Available: " - ] - }, - "_LEGACY_ERROR_TEMP_3255" : { - "message" : [ - "Error parsing '' to interval, " - ] - }, - "_LEGACY_ERROR_TEMP_3256" : { - "message" : [ - "Unrecognized datetime pattern: " - ] - }, - "_LEGACY_ERROR_TEMP_3257" : { - "message" : [ - "All week-based patterns are unsupported since Spark 3.0, detected: , Please use the SQL function EXTRACT instead" - ] - }, - "_LEGACY_ERROR_TEMP_3258" : { - "message" : [ - "Illegal pattern character: " - ] - }, - "_LEGACY_ERROR_TEMP_3259" : { - "message" : [ - "Too many pattern letters: