diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 0ebeea9aed8d2..0a9dcd52ea831 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1045,6 +1045,12 @@ ], "sqlState" : "42710" }, + "DATA_SOURCE_EXTERNAL_ERROR" : { + "message" : [ + "Encountered error when saving to external data source." + ], + "sqlState" : "KD00F" + }, "DATA_SOURCE_NOT_EXIST" : { "message" : [ "Data source '' not found. Please make sure the data source is registered." @@ -3118,12 +3124,12 @@ "subClass" : { "NOT_ALLOWED_IN_SCOPE" : { "message" : [ - "Variable was declared on line , which is not allowed in this scope." + "Declaration of the variable is not allowed in this scope." ] }, "ONLY_AT_BEGINNING" : { "message" : [ - "Variable can only be declared at the beginning of the compound, but it was declared on line ." + "Variable can only be declared at the beginning of the compound." ] } }, diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index c369db3f65058..edba6e1d43216 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -7417,6 +7417,12 @@ "standard": "N", "usedBy": ["Databricks"] }, + "KD00F": { + "description": "external data source failure", + "origin": "Databricks", + "standard": "N", + "usedBy": ["Databricks"] + }, "P0000": { "description": "procedural logic error", "origin": "PostgreSQL", diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala index 60685f5c0c6b9..700c05b54a256 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala @@ -77,8 +77,19 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest s"""CREATE TABLE pattern_testing_table ( |pattern_testing_col LONGTEXT |) - """.stripMargin + |""".stripMargin ).executeUpdate() + connection.prepareStatement( + "CREATE TABLE datetime (name VARCHAR(32), date1 DATE, time1 TIMESTAMP)") + .executeUpdate() + } + + override def dataPreparation(connection: Connection): Unit = { + super.dataPreparation(connection) + connection.prepareStatement("INSERT INTO datetime VALUES " + + "('amy', '2022-05-19', '2022-05-19 00:00:00')").executeUpdate() + connection.prepareStatement("INSERT INTO datetime VALUES " + + "('alex', '2022-05-18', '2022-05-18 00:00:00')").executeUpdate() } override def testUpdateColumnType(tbl: String): Unit = { @@ -157,6 +168,79 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest assert(sql(s"SELECT char_length(c1) from $tableName").head().get(0) === 65536) } } + + override def testDatetime(tbl: String): Unit = { + val df1 = sql(s"SELECT name FROM $tbl WHERE " + + "dayofyear(date1) > 100 AND dayofmonth(date1) > 10 ") + checkFilterPushed(df1) + val rows1 = df1.collect() + assert(rows1.length === 2) + assert(rows1(0).getString(0) === "amy") + assert(rows1(1).getString(0) === "alex") + + val df2 = sql(s"SELECT name FROM $tbl WHERE year(date1) = 2022 AND quarter(date1) = 2") + checkFilterPushed(df2) + val rows2 = df2.collect() + assert(rows2.length === 2) + assert(rows2(0).getString(0) === "amy") + assert(rows2(1).getString(0) === "alex") + + val df3 = sql(s"SELECT name FROM $tbl WHERE second(time1) = 0 AND month(date1) = 5") + checkFilterPushed(df3) + val rows3 = df3.collect() + assert(rows3.length === 2) + assert(rows3(0).getString(0) === "amy") + assert(rows3(1).getString(0) === "alex") + + val df4 = sql(s"SELECT name FROM $tbl WHERE hour(time1) = 0 AND minute(time1) = 0") + checkFilterPushed(df4) + val rows4 = df4.collect() + assert(rows4.length === 2) + assert(rows4(0).getString(0) === "amy") + assert(rows4(1).getString(0) === "alex") + + val df5 = sql(s"SELECT name FROM $tbl WHERE " + + "extract(WEEk from date1) > 10 AND extract(YEAROFWEEK from date1) = 2022") + checkFilterPushed(df5) + val rows5 = df5.collect() + assert(rows5.length === 2) + assert(rows5(0).getString(0) === "amy") + assert(rows5(1).getString(0) === "alex") + + val df6 = sql(s"SELECT name FROM $tbl WHERE date_add(date1, 1) = date'2022-05-20' " + + "AND datediff(date1, '2022-05-10') > 0") + checkFilterPushed(df6) + val rows6 = df6.collect() + assert(rows6.length === 1) + assert(rows6(0).getString(0) === "amy") + + val df7 = sql(s"SELECT name FROM $tbl WHERE weekday(date1) = 2") + checkFilterPushed(df7) + val rows7 = df7.collect() + assert(rows7.length === 1) + assert(rows7(0).getString(0) === "alex") + + val df8 = sql(s"SELECT name FROM $tbl WHERE dayofweek(date1) = 4") + checkFilterPushed(df8) + val rows8 = df8.collect() + assert(rows8.length === 1) + assert(rows8(0).getString(0) === "alex") + + val df9 = sql(s"SELECT name FROM $tbl WHERE " + + "dayofyear(date1) > 100 order by dayofyear(date1) limit 1") + checkFilterPushed(df9) + val rows9 = df9.collect() + assert(rows9.length === 1) + assert(rows9(0).getString(0) === "alex") + + // MySQL does not support + val df10 = sql(s"SELECT name FROM $tbl WHERE trunc(date1, 'week') = date'2022-05-16'") + checkFilterPushed(df10, false) + val rows10 = df10.collect() + assert(rows10.length === 2) + assert(rows10(0).getString(0) === "amy") + assert(rows10(1).getString(0) === "alex") + } } /** 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 d3629d871cd42..54635f69f8b65 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 @@ -353,7 +353,7 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu } } - private def checkFilterPushed(df: DataFrame, pushed: Boolean = true): Unit = { + protected def checkFilterPushed(df: DataFrame, pushed: Boolean = true): Unit = { val filter = df.queryExecution.optimizedPlan.collect { case f: Filter => f } @@ -980,4 +980,10 @@ private[v2] trait V2JDBCTest extends SharedSparkSession with DockerIntegrationFu ) } } + + def testDatetime(tbl: String): Unit = {} + + test("scan with filter push-down with date time functions") { + testDatetime(s"$catalogAndNamespace.${caseConvert("datetime")}") + } } diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 7106a780b3256..22c6280198c9a 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -27,7 +27,10 @@ trait SharedSparkContext extends BeforeAndAfterAll with BeforeAndAfterEach { sel def sc: SparkContext = _sc - val conf = new SparkConf(false) + // SPARK-49647: use `SparkConf()` instead of `SparkConf(false)` because we want to + // load defaults from system properties and the classpath, including default test + // settings specified in the SBT and Maven build definitions. + val conf: SparkConf = new SparkConf() /** * Initialize the [[SparkContext]]. Generally, this is just called from beforeAll; however, in diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index e86b91968bf80..e7f558b523d0c 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -94,7 +94,7 @@ ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library RUN add-apt-repository ppa:pypy/ppa RUN mkdir -p /usr/local/pypy/pypy3.9 && \ curl -sqL https://downloads.python.org/pypy/pypy3.9-v7.3.16-linux64.tar.bz2 | tar xjf - -C /usr/local/pypy/pypy3.9 --strip-components=1 && \ - ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3.8 && \ + ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3.9 && \ ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.2' scipy coverage matplotlib lxml diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 2db86ed229a01..e1ac039f25467 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -4,7 +4,7 @@ JTransforms/3.1//JTransforms-3.1.jar RoaringBitmap/1.2.1//RoaringBitmap-1.2.1.jar ST4/4.0.4//ST4-4.0.4.jar activation/1.1.1//activation-1.1.1.jar -aircompressor/0.27//aircompressor-0.27.jar +aircompressor/2.0.2//aircompressor-2.0.2.jar algebra_2.13/2.8.0//algebra_2.13-2.8.0.jar aliyun-java-sdk-core/4.5.10//aliyun-java-sdk-core-4.5.10.jar aliyun-java-sdk-kms/2.11.0//aliyun-java-sdk-kms-2.11.0.jar diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index ce47362999284..5939e429b2f35 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -88,7 +88,7 @@ ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library RUN add-apt-repository ppa:pypy/ppa RUN mkdir -p /usr/local/pypy/pypy3.9 && \ curl -sqL https://downloads.python.org/pypy/pypy3.9-v7.3.16-linux64.tar.bz2 | tar xjf - -C /usr/local/pypy/pypy3.9 --strip-components=1 && \ - ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3.8 && \ + ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3.9 && \ ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 RUN pypy3 -m pip install 'numpy==1.26.4' 'six==1.16.0' 'pandas==2.2.2' scipy coverage matplotlib lxml diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 34fbb8450d544..b9a4bed715f67 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -548,6 +548,8 @@ def __hash__(self): "pyspark.sql.tests.test_udtf", "pyspark.sql.tests.test_utils", "pyspark.sql.tests.test_resources", + "pyspark.sql.tests.plot.test_frame_plot", + "pyspark.sql.tests.plot.test_frame_plot_plotly", ], ) @@ -1051,6 +1053,8 @@ def __hash__(self): "pyspark.sql.tests.connect.test_parity_arrow_cogrouped_map", "pyspark.sql.tests.connect.test_parity_python_datasource", "pyspark.sql.tests.connect.test_parity_python_streaming_datasource", + "pyspark.sql.tests.connect.test_parity_frame_plot", + "pyspark.sql.tests.connect.test_parity_frame_plot_plotly", "pyspark.sql.tests.connect.test_utils", "pyspark.sql.tests.connect.client.test_artifact", "pyspark.sql.tests.connect.client.test_artifact_localcluster", diff --git a/docs/security.md b/docs/security.md index a8f4e4ec53897..b97abfeacf240 100644 --- a/docs/security.md +++ b/docs/security.md @@ -55,7 +55,8 @@ To enable authorization, Spark Master should have `spark.master.rest.filters=org.apache.spark.ui.JWSFilter` and `spark.org.apache.spark.ui.JWSFilter.param.secretKey=BASE64URL-ENCODED-KEY` configurations, and client should provide HTTP `Authorization` header which contains JSON Web Token signed by -the shared secret key. +the shared secret key. Please note that this feature requires a Spark distribution built with +`jjwt` profile. ### YARN @@ -813,6 +814,12 @@ They are generally private services, and should only be accessible within the ne organization that deploys Spark. Access to the hosts and ports used by Spark services should be limited to origin hosts that need to access the services. +However, like the REST Submission port, Spark also supports HTTP `Authorization` header +with a cryptographically signed JSON Web Token (JWT) for all UI ports. +To use it, a user needs the Spark distribution built with `jjwt` profile and to configure +`spark.ui.filters=org.apache.spark.ui.JWSFilter` and +`spark.org.apache.spark.ui.JWSFilter.param.secretKey=BASE64URL-ENCODED-KEY`. + Below are the primary ports that Spark uses for its communication and how to configure those ports. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index ad678c44657ed..0ecd45c2d8c56 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -60,6 +60,7 @@ license: | - Since Spark 4.0, By default views tolerate column type changes in the query and compensate with casts. To restore the previous behavior, allowing up-casts only, set `spark.sql.legacy.viewSchemaCompensation` to `false`. - Since Spark 4.0, Views allow control over how they react to underlying query changes. By default views tolerate column type changes in the query and compensate with casts. To disable this feature set `spark.sql.legacy.viewSchemaBindingMode` to `false`. This also removes the clause from `DESCRIBE EXTENDED` and `SHOW CREATE TABLE`. - Since Spark 4.0, The Storage-Partitioned Join feature flag `spark.sql.sources.v2.bucketing.pushPartValues.enabled` is set to `true`. To restore the previous behavior, set `spark.sql.sources.v2.bucketing.pushPartValues.enabled` to `false`. +- Since Spark 4.0, the `sentences` function uses `Locale(language)` instead of `Locale.US` when `language` parameter is not `NULL` and `country` parameter is `NULL`. ## Upgrading from Spark SQL 3.5.1 to 3.5.2 diff --git a/licenses-binary/LICENSE-xz.txt b/licenses-binary/LICENSE-xz.txt new file mode 100644 index 0000000000000..4322122aecf1a --- /dev/null +++ b/licenses-binary/LICENSE-xz.txt @@ -0,0 +1,11 @@ +Permission to use, copy, modify, and/or distribute this +software for any purpose with or without fee is hereby granted. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL +WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL +THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT, INDIRECT, OR +CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM +LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, +NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN +CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. diff --git a/pom.xml b/pom.xml index b1497c7826855..b9f28eb619258 100644 --- a/pom.xml +++ b/pom.xml @@ -2634,7 +2634,7 @@ io.airlift aircompressor - 0.27 + 2.0.2 org.apache.orc diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index 4061d024a83cd..92aeb15e21d1b 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1088,6 +1088,11 @@ "Function `` should use only POSITIONAL or POSITIONAL OR KEYWORD arguments." ] }, + "UNSUPPORTED_PLOT_BACKEND": { + "message": [ + "`` is not supported, it should be one of the values from " + ] + }, "UNSUPPORTED_SIGNATURE": { "message": [ "Unsupported signature: ." diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index 91b9591625904..d174f7774cc57 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -58,6 +58,7 @@ from pyspark.sql.classic.column import _to_seq, _to_list, _to_java_column from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.merge import MergeIntoWriter +from pyspark.sql.plot import PySparkPlotAccessor from pyspark.sql.streaming import DataStreamWriter from pyspark.sql.types import ( StructType, @@ -1862,6 +1863,10 @@ def executionInfo(self) -> Optional["ExecutionInfo"]: messageParameters={"member": "queryExecution"}, ) + @property + def plot(self) -> PySparkPlotAccessor: + return PySparkPlotAccessor(self) + class DataFrameNaFunctions(ParentDataFrameNaFunctions): def __init__(self, df: ParentDataFrame): diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 768abd655d497..e3b1d35b2d5d6 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -83,6 +83,7 @@ UnresolvedStar, ) from pyspark.sql.connect.functions import builtin as F +from pyspark.sql.plot import PySparkPlotAccessor from pyspark.sql.pandas.types import from_arrow_schema, to_arrow_schema from pyspark.sql.pandas.functions import _validate_pandas_udf # type: ignore[attr-defined] @@ -2239,6 +2240,10 @@ def rdd(self) -> "RDD[Row]": def executionInfo(self) -> Optional["ExecutionInfo"]: return self._execution_info + @property + def plot(self) -> PySparkPlotAccessor: + return PySparkPlotAccessor(self) + class DataFrameNaFunctions(ParentDataFrameNaFunctions): def __init__(self, df: ParentDataFrame): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ef35b73332572..7748510258eaa 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -39,6 +39,7 @@ from pyspark.sql.column import Column from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.merge import MergeIntoWriter +from pyspark.sql.plot import PySparkPlotAccessor from pyspark.sql.streaming import DataStreamWriter from pyspark.sql.types import StructType, Row from pyspark.sql.utils import dispatch_df_method @@ -6394,6 +6395,32 @@ def executionInfo(self) -> Optional["ExecutionInfo"]: """ ... + @property + def plot(self) -> PySparkPlotAccessor: + """ + Returns a :class:`PySparkPlotAccessor` for plotting functions. + + .. versionadded:: 4.0.0 + + Returns + ------- + :class:`PySparkPlotAccessor` + + Notes + ----- + This API is experimental. + + Examples + -------- + >>> data = [("A", 10, 1.5), ("B", 30, 2.5), ("C", 20, 3.5)] + >>> columns = ["category", "int_val", "float_val"] + >>> df = spark.createDataFrame(data, columns) + >>> type(df.plot) + + >>> df.plot.line(x="category", y=["int_val", "float_val"]) # doctest: +SKIP + """ + ... + class DataFrameNaFunctions: """Functionality for working with missing data in :class:`DataFrame`. diff --git a/python/pyspark/sql/plot/__init__.py b/python/pyspark/sql/plot/__init__.py new file mode 100644 index 0000000000000..6da07061b2a09 --- /dev/null +++ b/python/pyspark/sql/plot/__init__.py @@ -0,0 +1,21 @@ +# +# 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. +# + +""" +This package includes the plotting APIs for PySpark DataFrame. +""" +from pyspark.sql.plot.core import * # noqa: F403, F401 diff --git a/python/pyspark/sql/plot/core.py b/python/pyspark/sql/plot/core.py new file mode 100644 index 0000000000000..baee610dc6bd0 --- /dev/null +++ b/python/pyspark/sql/plot/core.py @@ -0,0 +1,135 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from typing import Any, TYPE_CHECKING, Optional, Union +from types import ModuleType +from pyspark.errors import PySparkRuntimeError, PySparkValueError +from pyspark.sql.utils import require_minimum_plotly_version + + +if TYPE_CHECKING: + from pyspark.sql import DataFrame + import pandas as pd + from plotly.graph_objs import Figure + + +class PySparkTopNPlotBase: + def get_top_n(self, sdf: "DataFrame") -> "pd.DataFrame": + from pyspark.sql import SparkSession + + session = SparkSession.getActiveSession() + if session is None: + raise PySparkRuntimeError(errorClass="NO_ACTIVE_SESSION", messageParameters=dict()) + + max_rows = int( + session.conf.get("spark.sql.pyspark.plotting.max_rows") # type: ignore[arg-type] + ) + pdf = sdf.limit(max_rows + 1).toPandas() + + self.partial = False + if len(pdf) > max_rows: + self.partial = True + pdf = pdf.iloc[:max_rows] + + return pdf + + +class PySparkSampledPlotBase: + def get_sampled(self, sdf: "DataFrame") -> "pd.DataFrame": + from pyspark.sql import SparkSession + + session = SparkSession.getActiveSession() + if session is None: + raise PySparkRuntimeError(errorClass="NO_ACTIVE_SESSION", messageParameters=dict()) + + sample_ratio = session.conf.get("spark.sql.pyspark.plotting.sample_ratio") + max_rows = int( + session.conf.get("spark.sql.pyspark.plotting.max_rows") # type: ignore[arg-type] + ) + + if sample_ratio is None: + fraction = 1 / (sdf.count() / max_rows) + fraction = min(1.0, fraction) + else: + fraction = float(sample_ratio) + + sampled_sdf = sdf.sample(fraction=fraction) + pdf = sampled_sdf.toPandas() + + return pdf + + +class PySparkPlotAccessor: + plot_data_map = { + "line": PySparkSampledPlotBase().get_sampled, + } + _backends = {} # type: ignore[var-annotated] + + def __init__(self, data: "DataFrame"): + self.data = data + + def __call__( + self, kind: str = "line", backend: Optional[str] = None, **kwargs: Any + ) -> "Figure": + plot_backend = PySparkPlotAccessor._get_plot_backend(backend) + + return plot_backend.plot_pyspark(self.data, kind=kind, **kwargs) + + @staticmethod + def _get_plot_backend(backend: Optional[str] = None) -> ModuleType: + backend = backend or "plotly" + + if backend in PySparkPlotAccessor._backends: + return PySparkPlotAccessor._backends[backend] + + if backend == "plotly": + require_minimum_plotly_version() + else: + raise PySparkValueError( + errorClass="UNSUPPORTED_PLOT_BACKEND", + messageParameters={"backend": backend, "supported_backends": ", ".join(["plotly"])}, + ) + from pyspark.sql.plot import plotly as module + + return module + + def line(self, x: str, y: Union[str, list[str]], **kwargs: Any) -> "Figure": + """ + Plot DataFrame as lines. + + Parameters + ---------- + x : str + Name of column to use for the horizontal axis. + y : str or list of str + Name(s) of the column(s) to use for the vertical axis. Multiple columns can be plotted. + **kwds : optional + Additional keyword arguments. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + + Examples + -------- + >>> data = [("A", 10, 1.5), ("B", 30, 2.5), ("C", 20, 3.5)] + >>> columns = ["category", "int_val", "float_val"] + >>> df = spark.createDataFrame(data, columns) + >>> df.plot.line(x="category", y="int_val") # doctest: +SKIP + >>> df.plot.line(x="category", y=["int_val", "float_val"]) # doctest: +SKIP + """ + return self(kind="line", x=x, y=y, **kwargs) diff --git a/python/pyspark/sql/plot/plotly.py b/python/pyspark/sql/plot/plotly.py new file mode 100644 index 0000000000000..5efc19476057f --- /dev/null +++ b/python/pyspark/sql/plot/plotly.py @@ -0,0 +1,30 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from typing import TYPE_CHECKING, Any + +from pyspark.sql.plot import PySparkPlotAccessor + +if TYPE_CHECKING: + from pyspark.sql import DataFrame + from plotly.graph_objs import Figure + + +def plot_pyspark(data: "DataFrame", kind: str, **kwargs: Any) -> "Figure": + import plotly + + return plotly.plot(PySparkPlotAccessor.plot_data_map[kind](data), kind, **kwargs) diff --git a/python/pyspark/sql/tests/connect/test_parity_frame_plot.py b/python/pyspark/sql/tests/connect/test_parity_frame_plot.py new file mode 100644 index 0000000000000..c69e438bf7eb0 --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_frame_plot.py @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.testing.connectutils import ReusedConnectTestCase +from pyspark.sql.tests.plot.test_frame_plot import DataFramePlotTestsMixin + + +class FramePlotParityTests(DataFramePlotTestsMixin, ReusedConnectTestCase): + pass + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.connect.test_parity_frame_plot import * # noqa: F401 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/connect/test_parity_frame_plot_plotly.py b/python/pyspark/sql/tests/connect/test_parity_frame_plot_plotly.py new file mode 100644 index 0000000000000..78508fe533379 --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_frame_plot_plotly.py @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.testing.connectutils import ReusedConnectTestCase +from pyspark.sql.tests.plot.test_frame_plot_plotly import DataFramePlotPlotlyTestsMixin + + +class FramePlotPlotlyParityTests(DataFramePlotPlotlyTestsMixin, ReusedConnectTestCase): + pass + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.connect.test_parity_frame_plot_plotly import * # noqa: F401 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/plot/__init__.py b/python/pyspark/sql/tests/plot/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/sql/tests/plot/__init__.py @@ -0,0 +1,16 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# diff --git a/python/pyspark/sql/tests/plot/test_frame_plot.py b/python/pyspark/sql/tests/plot/test_frame_plot.py new file mode 100644 index 0000000000000..19ef53e46b2f4 --- /dev/null +++ b/python/pyspark/sql/tests/plot/test_frame_plot.py @@ -0,0 +1,79 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.errors import PySparkValueError +from pyspark.sql import Row +from pyspark.sql.plot import PySparkSampledPlotBase, PySparkTopNPlotBase +from pyspark.testing.sqlutils import ReusedSQLTestCase + + +class DataFramePlotTestsMixin: + def test_backend(self): + accessor = self.spark.range(2).plot + backend = accessor._get_plot_backend() + self.assertEqual(backend.__name__, "pyspark.sql.plot.plotly") + + with self.assertRaises(PySparkValueError) as pe: + accessor._get_plot_backend("matplotlib") + + self.check_error( + exception=pe.exception, + errorClass="UNSUPPORTED_PLOT_BACKEND", + messageParameters={"backend": "matplotlib", "supported_backends": "plotly"}, + ) + + def test_topn_max_rows(self): + try: + self.spark.conf.set("spark.sql.pyspark.plotting.max_rows", "1000") + sdf = self.spark.range(2500) + pdf = PySparkTopNPlotBase().get_top_n(sdf) + self.assertEqual(len(pdf), 1000) + finally: + self.spark.conf.unset("spark.sql.pyspark.plotting.max_rows") + + def test_sampled_plot_with_ratio(self): + try: + self.spark.conf.set("spark.sql.pyspark.plotting.sample_ratio", "0.5") + data = [Row(a=i, b=i + 1, c=i + 2, d=i + 3) for i in range(2500)] + sdf = self.spark.createDataFrame(data) + pdf = PySparkSampledPlotBase().get_sampled(sdf) + self.assertEqual(round(len(pdf) / 2500, 1), 0.5) + finally: + self.spark.conf.unset("spark.sql.pyspark.plotting.sample_ratio") + + def test_sampled_plot_with_max_rows(self): + data = [Row(a=i, b=i + 1, c=i + 2, d=i + 3) for i in range(2000)] + sdf = self.spark.createDataFrame(data) + pdf = PySparkSampledPlotBase().get_sampled(sdf) + self.assertEqual(round(len(pdf) / 2000, 1), 0.5) + + +class DataFramePlotTests(DataFramePlotTestsMixin, ReusedSQLTestCase): + pass + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.plot.test_frame_plot import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py new file mode 100644 index 0000000000000..72a3ed267d192 --- /dev/null +++ b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py @@ -0,0 +1,64 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import unittest +import pyspark.sql.plot # noqa: F401 +from pyspark.testing.sqlutils import ReusedSQLTestCase, have_plotly, plotly_requirement_message + + +@unittest.skipIf(not have_plotly, plotly_requirement_message) +class DataFramePlotPlotlyTestsMixin: + @property + def sdf(self): + data = [("A", 10, 1.5), ("B", 30, 2.5), ("C", 20, 3.5)] + columns = ["category", "int_val", "float_val"] + return self.spark.createDataFrame(data, columns) + + def _check_fig_data(self, fig_data, expected_x, expected_y, expected_name=""): + self.assertEqual(fig_data["mode"], "lines") + self.assertEqual(fig_data["type"], "scatter") + self.assertEqual(fig_data["xaxis"], "x") + self.assertEqual(list(fig_data["x"]), expected_x) + self.assertEqual(fig_data["yaxis"], "y") + self.assertEqual(list(fig_data["y"]), expected_y) + self.assertEqual(fig_data["name"], expected_name) + + def test_line_plot(self): + # single column as vertical axis + fig = self.sdf.plot(kind="line", x="category", y="int_val") + self._check_fig_data(fig["data"][0], ["A", "B", "C"], [10, 30, 20]) + + # multiple columns as vertical axis + fig = self.sdf.plot.line(x="category", y=["int_val", "float_val"]) + self._check_fig_data(fig["data"][0], ["A", "B", "C"], [10, 30, 20], "int_val") + self._check_fig_data(fig["data"][1], ["A", "B", "C"], [1.5, 2.5, 3.5], "float_val") + + +class DataFramePlotPlotlyTests(DataFramePlotPlotlyTestsMixin, ReusedSQLTestCase): + pass + + +if __name__ == "__main__": + from pyspark.sql.tests.plot.test_frame_plot_plotly import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/test_creation.py b/python/pyspark/sql/tests/test_creation.py index dfe66cdd3edf0..c6917aa234b41 100644 --- a/python/pyspark/sql/tests/test_creation.py +++ b/python/pyspark/sql/tests/test_creation.py @@ -15,7 +15,6 @@ # limitations under the License. # -import platform from decimal import Decimal import os import time @@ -111,11 +110,7 @@ def test_create_dataframe_from_pandas_with_dst(self): os.environ["TZ"] = orig_env_tz time.tzset() - # TODO(SPARK-43354): Re-enable test_create_dataframe_from_pandas_with_day_time_interval - @unittest.skipIf( - "pypy" in platform.python_implementation().lower() or not have_pandas, - "Fails in PyPy Python 3.8, should enable.", - ) + @unittest.skipIf(not have_pandas, pandas_requirement_message) # type: ignore def test_create_dataframe_from_pandas_with_day_time_interval(self): # SPARK-37277: Test DayTimeIntervalType in createDataFrame without Arrow. import pandas as pd diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 11b91612419a3..5d9ec92cbc830 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -41,6 +41,7 @@ PythonException, UnknownException, SparkUpgradeException, + PySparkImportError, PySparkNotImplementedError, PySparkRuntimeError, ) @@ -115,6 +116,22 @@ def require_test_compiled() -> None: ) +def require_minimum_plotly_version() -> None: + """Raise ImportError if plotly is not installed""" + minimum_plotly_version = "4.8" + + try: + import plotly # noqa: F401 + except ImportError as error: + raise PySparkImportError( + errorClass="PACKAGE_NOT_INSTALLED", + messageParameters={ + "package_name": "plotly", + "minimum_version": str(minimum_plotly_version), + }, + ) from error + + class ForeachBatchFunction: """ This is the Python implementation of Java interface 'ForeachBatchFunction'. This wraps diff --git a/python/pyspark/testing/sqlutils.py b/python/pyspark/testing/sqlutils.py index 9f07c44c084cf..00ad40e68bd7c 100644 --- a/python/pyspark/testing/sqlutils.py +++ b/python/pyspark/testing/sqlutils.py @@ -48,6 +48,13 @@ except Exception as e: test_not_compiled_message = str(e) +plotly_requirement_message = None +try: + import plotly +except ImportError as e: + plotly_requirement_message = str(e) +have_plotly = plotly_requirement_message is None + from pyspark.sql import SparkSession from pyspark.sql.types import ArrayType, DoubleType, UserDefinedType, Row from pyspark.testing.utils import ReusedPySparkTestCase, PySparkErrorTestUtils 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 42f0094de3515..73d5cb55295ab 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 @@ -64,6 +64,7 @@ compoundStatement | setStatementWithOptionalVarKeyword | beginEndCompoundBlock | ifElseStatement + | caseStatement | whileStatement | repeatStatement | leaveStatement @@ -98,6 +99,13 @@ iterateStatement : ITERATE multipartIdentifier ; +caseStatement + : CASE (WHEN conditions+=booleanExpression THEN conditionalBodies+=compoundBody)+ + (ELSE elseBody=compoundBody)? END CASE #searchedCaseStatement + | CASE caseVariable=expression (WHEN conditionExpressions+=expression THEN conditionalBodies+=compoundBody)+ + (ELSE elseBody=compoundBody)? END CASE #simpleCaseStatement + ; + singleStatement : (statement|setResetStatement) SEMICOLON* EOF ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index d0ee9f2d110d5..3cdde622d51f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -79,7 +79,7 @@ object ConstantFolding extends Rule[LogicalPlan] { // Fold expressions that are foldable. case e if e.foldable => try { - Literal.create(e.eval(EmptyRow), e.dataType) + Literal.create(e.freshCopyIfContainsStatefulExpression().eval(EmptyRow), e.dataType) } catch { case NonFatal(_) if isConditionalBranch => // When doing constant folding inside conditional expressions, we should not fail 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 924b5c2cfeb15..7ad7d60e70c96 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 @@ -173,14 +173,10 @@ class AstBuilder extends DataTypeAstBuilder case Some(c: CreateVariable) => if (allowVarDeclare) { throw SqlScriptingErrors.variableDeclarationOnlyAtBeginning( - c.origin, - toSQLId(c.name.asInstanceOf[UnresolvedIdentifier].nameParts), - c.origin.line.get.toString) + c.origin, c.name.asInstanceOf[UnresolvedIdentifier].nameParts) } else { throw SqlScriptingErrors.variableDeclarationNotAllowedInScope( - c.origin, - toSQLId(c.name.asInstanceOf[UnresolvedIdentifier].nameParts), - c.origin.line.get.toString) + c.origin, c.name.asInstanceOf[UnresolvedIdentifier].nameParts) } case _ => } @@ -200,7 +196,9 @@ class AstBuilder extends DataTypeAstBuilder el.multipartIdentifier().getText.toLowerCase(Locale.ROOT) => withOrigin(bl) { throw SqlScriptingErrors.labelsMismatch( - CurrentOrigin.get, bl.multipartIdentifier().getText, el.multipartIdentifier().getText) + CurrentOrigin.get, + bl.multipartIdentifier().getText, + el.multipartIdentifier().getText) } case (None, Some(el: EndLabelContext)) => withOrigin(el) { @@ -261,6 +259,52 @@ class AstBuilder extends DataTypeAstBuilder WhileStatement(condition, body, Some(labelText)) } + override def visitSearchedCaseStatement(ctx: SearchedCaseStatementContext): CaseStatement = { + val conditions = ctx.conditions.asScala.toList.map(boolExpr => withOrigin(boolExpr) { + SingleStatement( + Project( + Seq(Alias(expression(boolExpr), "condition")()), + OneRowRelation())) + }) + val conditionalBodies = + ctx.conditionalBodies.asScala.toList.map(body => visitCompoundBody(body)) + + if (conditions.length != conditionalBodies.length) { + throw SparkException.internalError( + s"Mismatched number of conditions ${conditions.length} and condition bodies" + + s" ${conditionalBodies.length} in case statement") + } + + CaseStatement( + conditions = conditions, + conditionalBodies = conditionalBodies, + elseBody = Option(ctx.elseBody).map(body => visitCompoundBody(body))) + } + + override def visitSimpleCaseStatement(ctx: SimpleCaseStatementContext): CaseStatement = { + // uses EqualTo to compare the case variable(the main case expression) + // to the WHEN clause expressions + val conditions = ctx.conditionExpressions.asScala.toList.map(expr => withOrigin(expr) { + SingleStatement( + Project( + Seq(Alias(EqualTo(expression(ctx.caseVariable), expression(expr)), "condition")()), + OneRowRelation())) + }) + val conditionalBodies = + ctx.conditionalBodies.asScala.toList.map(body => visitCompoundBody(body)) + + if (conditions.length != conditionalBodies.length) { + throw SparkException.internalError( + s"Mismatched number of conditions ${conditions.length} and condition bodies" + + s" ${conditionalBodies.length} in case statement") + } + + CaseStatement( + conditions = conditions, + conditionalBodies = conditionalBodies, + elseBody = Option(ctx.elseBody).map(body => visitCompoundBody(body))) + } + override def visitRepeatStatement(ctx: RepeatStatementContext): RepeatStatement = { val labelText = generateLabelText(Option(ctx.beginLabel()), Option(ctx.endLabel())) val boolExpr = ctx.booleanExpression() @@ -292,7 +336,7 @@ class AstBuilder extends DataTypeAstBuilder case c: RepeatStatementContext if Option(c.beginLabel()).isDefined && c.beginLabel().multipartIdentifier().getText.toLowerCase(Locale.ROOT).equals(label) - => true + => true case _ => false } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala index 5e7e8b0b4fc9a..ed40a5fd734b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala @@ -124,3 +124,17 @@ case class LeaveStatement(label: String) extends CompoundPlanStatement * @param label Label of the loop to iterate. */ case class IterateStatement(label: String) extends CompoundPlanStatement + +/** + * Logical operator for CASE statement. + * @param conditions Collection of conditions which correspond to WHEN clauses. + * @param conditionalBodies Collection of bodies that have a corresponding condition, + * in WHEN branches. + * @param elseBody Body that is executed if none of the conditions are met, i.e. ELSE branch. + */ +case class CaseStatement( + conditions: Seq[SingleStatement], + conditionalBodies: Seq[CompoundBody], + elseBody: Option[CompoundBody]) extends CompoundPlanStatement { + assert(conditions.length == conditionalBodies.length) +} 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 fa8ea2f5289fa..e4c8c76e958f8 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 @@ -3959,6 +3959,14 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map("provider" -> name)) } + def externalDataSourceException(cause: Throwable): Throwable = { + new AnalysisException( + errorClass = "DATA_SOURCE_EXTERNAL_ERROR", + messageParameters = Map(), + cause = Some(cause) + ) + } + def foundMultipleDataSources(provider: String): Throwable = { new AnalysisException( errorClass = "FOUND_MULTIPLE_DATA_SOURCES", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala index 591d2e3e53d47..7f13dc334e06e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.errors import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryExecutionErrors.toSQLStmt import org.apache.spark.sql.exceptions.SqlScriptingException @@ -32,7 +33,7 @@ private[sql] object SqlScriptingErrors { origin = origin, errorClass = "LABELS_MISMATCH", cause = null, - messageParameters = Map("beginLabel" -> beginLabel, "endLabel" -> endLabel)) + messageParameters = Map("beginLabel" -> toSQLId(beginLabel), "endLabel" -> toSQLId(endLabel))) } def endLabelWithoutBeginLabel(origin: Origin, endLabel: String): Throwable = { @@ -40,29 +41,27 @@ private[sql] object SqlScriptingErrors { origin = origin, errorClass = "END_LABEL_WITHOUT_BEGIN_LABEL", cause = null, - messageParameters = Map("endLabel" -> endLabel)) + messageParameters = Map("endLabel" -> toSQLId(endLabel))) } def variableDeclarationNotAllowedInScope( origin: Origin, - varName: String, - lineNumber: String): Throwable = { + varName: Seq[String]): Throwable = { new SqlScriptingException( origin = origin, errorClass = "INVALID_VARIABLE_DECLARATION.NOT_ALLOWED_IN_SCOPE", cause = null, - messageParameters = Map("varName" -> varName, "lineNumber" -> lineNumber)) + messageParameters = Map("varName" -> toSQLId(varName))) } def variableDeclarationOnlyAtBeginning( origin: Origin, - varName: String, - lineNumber: String): Throwable = { + varName: Seq[String]): Throwable = { new SqlScriptingException( origin = origin, errorClass = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING", cause = null, - messageParameters = Map("varName" -> varName, "lineNumber" -> lineNumber)) + messageParameters = Map("varName" -> toSQLId(varName))) } def invalidBooleanStatement( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala index 4354e7e3635e4..f0c28c95046eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.exceptions.SqlScriptingException.errorMessageWithLin class SqlScriptingException ( errorClass: String, cause: Throwable, - origin: Origin, + val origin: Origin, messageParameters: Map[String, String] = Map.empty) extends Exception( errorMessageWithLineNumber(Option(origin), errorClass, messageParameters), 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 a87b0613292c9..5853e4b66dcc0 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 @@ -3169,6 +3169,29 @@ object SQLConf { .version("4.0.0") .fallbackConf(Python.PYTHON_WORKER_FAULTHANLDER_ENABLED) + val PYSPARK_PLOT_MAX_ROWS = + buildConf("spark.sql.pyspark.plotting.max_rows") + .doc( + "The visual limit on top-n-based plots. If set to 1000, the first 1000 data points " + + "will be used for plotting.") + .version("4.0.0") + .intConf + .createWithDefault(1000) + + val PYSPARK_PLOT_SAMPLE_RATIO = + buildConf("spark.sql.pyspark.plotting.sample_ratio") + .doc( + "The proportion of data that will be plotted for sample-based plots. It is determined " + + "based on spark.sql.pyspark.plotting.max_rows if not explicitly set." + ) + .version("4.0.0") + .doubleConf + .checkValue( + ratio => ratio >= 0.0 && ratio <= 1.0, + "The value should be between 0.0 and 1.0 inclusive." + ) + .createOptional + val ARROW_SPARKR_EXECUTION_ENABLED = buildConf("spark.sql.execution.arrow.sparkr.enabled") .doc("When true, make use of Apache Arrow for columnar data transfers in SparkR. " + @@ -5855,6 +5878,10 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def pythonUDFWorkerFaulthandlerEnabled: Boolean = getConf(PYTHON_UDF_WORKER_FAULTHANLDER_ENABLED) + def pysparkPlotMaxRows: Int = getConf(PYSPARK_PLOT_MAX_ROWS) + + def pysparkPlotSampleRatio: Option[Double] = getConf(PYSPARK_PLOT_SAMPLE_RATIO) + def arrowSparkREnabled: Boolean = getConf(ARROW_SPARKR_EXECUTION_ENABLED) def arrowPySparkFallbackEnabled: Boolean = getConf(ARROW_PYSPARK_FALLBACK_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index bf527b9c3bd7d..ba634333e06fb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{Alias, EqualTo, Expression, In, Literal, ScalarSubquery} import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.catalyst.plans.logical.CreateVariable +import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, Project} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.exceptions.SqlScriptingException class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { @@ -205,13 +207,14 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT a, b, c FROM T; | SELECT * FROM T; |END lbl_end""".stripMargin - + val exception = intercept[SqlScriptingException] { + parseScript(sqlScriptText) + } checkError( - exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) - }, + exception = exception, condition = "LABELS_MISMATCH", - parameters = Map("beginLabel" -> "lbl_begin", "endLabel" -> "lbl_end")) + parameters = Map("beginLabel" -> toSQLId("lbl_begin"), "endLabel" -> toSQLId("lbl_end"))) + assert(exception.origin.line.contains(2)) } test("compound: endLabel") { @@ -224,13 +227,14 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT a, b, c FROM T; | SELECT * FROM T; |END lbl""".stripMargin - + val exception = intercept[SqlScriptingException] { + parseScript(sqlScriptText) + } checkError( - exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) - }, + exception = exception, condition = "END_LABEL_WITHOUT_BEGIN_LABEL", - parameters = Map("endLabel" -> "lbl")) + parameters = Map("endLabel" -> toSQLId("lbl"))) + assert(exception.origin.line.contains(8)) } test("compound: beginLabel + endLabel with different casing") { @@ -286,12 +290,14 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT 1; | DECLARE testVariable INTEGER; |END""".stripMargin + val exception = intercept[SqlScriptingException] { + parseScript(sqlScriptText) + } checkError( - exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) - }, + exception = exception, condition = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING", - parameters = Map("varName" -> "`testVariable`", "lineNumber" -> "4")) + parameters = Map("varName" -> "`testVariable`")) + assert(exception.origin.line.contains(4)) } test("declare in wrong scope") { @@ -302,12 +308,14 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | DECLARE testVariable INTEGER; | END IF; |END""".stripMargin + val exception = intercept[SqlScriptingException] { + parseScript(sqlScriptText) + } checkError( - exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) - }, + exception = exception, condition = "INVALID_VARIABLE_DECLARATION.NOT_ALLOWED_IN_SCOPE", - parameters = Map("varName" -> "`testVariable`", "lineNumber" -> "4")) + parameters = Map("varName" -> "`testVariable`")) + assert(exception.origin.line.contains(4)) } test("SET VAR statement test") { @@ -1111,6 +1119,287 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { } + test("searched case statement") { + val sqlScriptText = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.conditions.length == 1) + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditions.head.getText == "1 = 1") + } + + test("searched case statement - multi when") { + val sqlScriptText = + """ + |BEGIN + | CASE + | WHEN 1 IN (1,2,3) THEN + | SELECT 1; + | WHEN (SELECT * FROM t) THEN + | SELECT * FROM b; + | WHEN 1 = 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.conditions.length == 3) + assert(caseStmt.conditionalBodies.length == 3) + assert(caseStmt.elseBody.isEmpty) + + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditions.head.getText == "1 IN (1,2,3)") + + assert(caseStmt.conditionalBodies.head.collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditionalBodies.head.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 1") + + assert(caseStmt.conditions(1).isInstanceOf[SingleStatement]) + assert(caseStmt.conditions(1).getText == "(SELECT * FROM t)") + + assert(caseStmt.conditionalBodies(1).collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditionalBodies(1).collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT * FROM b") + + assert(caseStmt.conditions(2).isInstanceOf[SingleStatement]) + assert(caseStmt.conditions(2).getText == "1 = 1") + + assert(caseStmt.conditionalBodies(2).collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditionalBodies(2).collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 42") + } + + test("searched case statement with else") { + val sqlScriptText = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.elseBody.isDefined) + assert(caseStmt.conditions.length == 1) + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditions.head.getText == "1 = 1") + + assert(caseStmt.elseBody.get.collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.elseBody.get.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 43") + } + + test("searched case statement nested") { + val sqlScriptText = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | CASE + | WHEN 2 = 1 THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END CASE; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.conditions.length == 1) + assert(caseStmt.conditionalBodies.length == 1) + assert(caseStmt.elseBody.isEmpty) + + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditions.head.getText == "1 = 1") + + assert(caseStmt.conditionalBodies.head.collection.head.isInstanceOf[CaseStatement]) + val nestedCaseStmt = + caseStmt.conditionalBodies.head.collection.head.asInstanceOf[CaseStatement] + + assert(nestedCaseStmt.conditions.length == 1) + assert(nestedCaseStmt.conditionalBodies.length == 1) + assert(nestedCaseStmt.elseBody.isDefined) + + assert(nestedCaseStmt.conditions.head.isInstanceOf[SingleStatement]) + assert(nestedCaseStmt.conditions.head.getText == "2 = 1") + + assert(nestedCaseStmt.conditionalBodies.head.collection.head.isInstanceOf[SingleStatement]) + assert(nestedCaseStmt.conditionalBodies.head.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 41") + + assert(nestedCaseStmt.elseBody.get.collection.head.isInstanceOf[SingleStatement]) + assert(nestedCaseStmt.elseBody.get.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 42") + } + + test("simple case statement") { + val sqlScriptText = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | SELECT 1; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.conditions.length == 1) + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition(caseStmt.conditions.head, _ == Literal(1), _ == Literal(1)) + } + + + test("simple case statement - multi when") { + val sqlScriptText = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | SELECT 1; + | WHEN (SELECT 2) THEN + | SELECT * FROM b; + | WHEN 3 IN (1,2,3) THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.conditions.length == 3) + assert(caseStmt.conditionalBodies.length == 3) + assert(caseStmt.elseBody.isEmpty) + + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition(caseStmt.conditions.head, _ == Literal(1), _ == Literal(1)) + + assert(caseStmt.conditionalBodies.head.collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditionalBodies.head.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 1") + + assert(caseStmt.conditions(1).isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition( + caseStmt.conditions(1), _ == Literal(1), _.isInstanceOf[ScalarSubquery]) + + assert(caseStmt.conditionalBodies(1).collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditionalBodies(1).collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT * FROM b") + + assert(caseStmt.conditions(2).isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition( + caseStmt.conditions(2), _ == Literal(1), _.isInstanceOf[In]) + + assert(caseStmt.conditionalBodies(2).collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.conditionalBodies(2).collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 42") + } + + test("simple case statement with else") { + val sqlScriptText = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.elseBody.isDefined) + assert(caseStmt.conditions.length == 1) + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition(caseStmt.conditions.head, _ == Literal(1), _ == Literal(1)) + + assert(caseStmt.elseBody.get.collection.head.isInstanceOf[SingleStatement]) + assert(caseStmt.elseBody.get.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 43") + } + + test("simple case statement nested") { + val sqlScriptText = + """ + |BEGIN + | CASE (SELECT 1) + | WHEN 1 THEN + | CASE 2 + | WHEN 2 THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END CASE; + | END CASE; + |END + |""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 1) + assert(tree.collection.head.isInstanceOf[CaseStatement]) + + val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] + assert(caseStmt.conditions.length == 1) + assert(caseStmt.conditionalBodies.length == 1) + assert(caseStmt.elseBody.isEmpty) + + assert(caseStmt.conditions.head.isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition( + caseStmt.conditions.head, _.isInstanceOf[ScalarSubquery], _ == Literal(1)) + + assert(caseStmt.conditionalBodies.head.collection.head.isInstanceOf[CaseStatement]) + val nestedCaseStmt = + caseStmt.conditionalBodies.head.collection.head.asInstanceOf[CaseStatement] + + assert(nestedCaseStmt.conditions.length == 1) + assert(nestedCaseStmt.conditionalBodies.length == 1) + assert(nestedCaseStmt.elseBody.isDefined) + + assert(nestedCaseStmt.conditions.head.isInstanceOf[SingleStatement]) + checkSimpleCaseStatementCondition( + nestedCaseStmt.conditions.head, _ == Literal(2), _ == Literal(2)) + + assert(nestedCaseStmt.conditionalBodies.head.collection.head.isInstanceOf[SingleStatement]) + assert(nestedCaseStmt.conditionalBodies.head.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 41") + + assert(nestedCaseStmt.elseBody.get.collection.head.isInstanceOf[SingleStatement]) + assert(nestedCaseStmt.elseBody.get.collection.head.asInstanceOf[SingleStatement] + .getText == "SELECT 42") + } + // Helper methods def cleanupStatementString(statementStr: String): String = { statementStr @@ -1119,4 +1408,17 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { .replace("END", "") .trim } + + private def checkSimpleCaseStatementCondition( + conditionStatement: SingleStatement, + predicateLeft: Expression => Boolean, + predicateRight: Expression => Boolean): Unit = { + assert(conditionStatement.parsedPlan.isInstanceOf[Project]) + val project = conditionStatement.parsedPlan.asInstanceOf[Project] + assert(project.projectList.head.isInstanceOf[Alias]) + assert(project.projectList.head.asInstanceOf[Alias].child.isInstanceOf[EqualTo]) + val equalTo = project.projectList.head.asInstanceOf[Alias].child.asInstanceOf[EqualTo] + assert(predicateLeft(equalTo.left)) + assert(predicateRight(equalTo.right)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala index 5423232db4293..e44f1d35e9cdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution.datasources import scala.util.control.NonFatal +import org.apache.spark.SparkThrowable import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{CTEInChildren, CTERelationDef, LogicalPlan, WithCTE} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.sources.CreatableRelationProvider +import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider} /** * Saves the results of `query` in to a data source. @@ -44,8 +46,26 @@ case class SaveIntoDataSourceCommand( override def innerChildren: Seq[QueryPlan[_]] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { - val relation = dataSource.createRelation( - sparkSession.sqlContext, mode, options, Dataset.ofRows(sparkSession, query)) + var relation: BaseRelation = null + + try { + relation = dataSource.createRelation( + sparkSession.sqlContext, mode, options, Dataset.ofRows(sparkSession, query)) + } catch { + case e: SparkThrowable => + // We should avoid wrapping `SparkThrowable` exceptions into another `AnalysisException`. + throw e + case e @ (_: NullPointerException | _: MatchError | _: ArrayIndexOutOfBoundsException) => + // These are some of the exceptions thrown by the data source API. We catch these + // exceptions here and rethrow QueryCompilationErrors.externalDataSourceException to + // provide a more friendly error message for the user. This list is not exhaustive. + throw QueryCompilationErrors.externalDataSourceException(e) + case e: Throwable => + // For other exceptions, just rethrow it, since we don't have enough information to + // provide a better error message for the user at the moment. We may want to further + // improve the error message handling in the future. + throw e + } try { val logicalRelation = LogicalRelation(relation, toAttributes(relation.schema), None, false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala index f2b626490d13c..785bf5b13aa78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala @@ -46,12 +46,33 @@ private case class MySQLDialect() extends JdbcDialect with SQLConfHelper with No // See https://dev.mysql.com/doc/refman/8.0/en/aggregate-functions.html private val supportedAggregateFunctions = Set("MAX", "MIN", "SUM", "COUNT", "AVG") ++ distinctUnsupportedAggregateFunctions - private val supportedFunctions = supportedAggregateFunctions + private val supportedFunctions = supportedAggregateFunctions ++ Set("DATE_ADD", "DATE_DIFF") override def isSupportedFunction(funcName: String): Boolean = supportedFunctions.contains(funcName) class MySQLSQLBuilder extends JDBCSQLBuilder { + override def visitExtract(field: String, source: String): String = { + field match { + case "DAY_OF_YEAR" => s"DAYOFYEAR($source)" + case "YEAR_OF_WEEK" => s"EXTRACT(YEAR FROM $source)" + // WEEKDAY uses Monday = 0, Tuesday = 1, ... and ISO standard is Monday = 1, ..., + // so we use the formula (WEEKDAY + 1) to follow the ISO standard. + case "DAY_OF_WEEK" => s"(WEEKDAY($source) + 1)" + case _ => super.visitExtract(field, source) + } + } + + override def visitSQLFunction(funcName: String, inputs: Array[String]): String = { + funcName match { + case "DATE_ADD" => + s"DATE_ADD(${inputs(0)}, INTERVAL ${inputs(1)} DAY)" + case "DATE_DIFF" => + s"DATEDIFF(${inputs(0)}, ${inputs(1)})" + case _ => super.visitSQLFunction(funcName, inputs) + } + } + override def visitSortOrder( sortKey: String, sortDirection: SortDirection, nullOrdering: NullOrdering): String = { (sortDirection, nullOrdering) match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index cae7976143142..af9fd5464277c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -405,6 +405,78 @@ class WhileStatementExec( } } +/** + * Executable node for CaseStatement. + * @param conditions Collection of executable conditions which correspond to WHEN clauses. + * @param conditionalBodies Collection of executable bodies that have a corresponding condition, + * in WHEN branches. + * @param elseBody Body that is executed if none of the conditions are met, i.e. ELSE branch. + * @param session Spark session that SQL script is executed within. + */ +class CaseStatementExec( + conditions: Seq[SingleStatementExec], + conditionalBodies: Seq[CompoundBodyExec], + elseBody: Option[CompoundBodyExec], + session: SparkSession) extends NonLeafStatementExec { + private object CaseState extends Enumeration { + val Condition, Body = Value + } + + private var state = CaseState.Condition + private var curr: Option[CompoundStatementExec] = Some(conditions.head) + + private var clauseIdx: Int = 0 + private val conditionsCount = conditions.length + + private lazy val treeIterator: Iterator[CompoundStatementExec] = + new Iterator[CompoundStatementExec] { + override def hasNext: Boolean = curr.nonEmpty + + override def next(): CompoundStatementExec = state match { + case CaseState.Condition => + val condition = curr.get.asInstanceOf[SingleStatementExec] + if (evaluateBooleanCondition(session, condition)) { + state = CaseState.Body + curr = Some(conditionalBodies(clauseIdx)) + } else { + clauseIdx += 1 + if (clauseIdx < conditionsCount) { + // There are WHEN clauses remaining. + state = CaseState.Condition + curr = Some(conditions(clauseIdx)) + } else if (elseBody.isDefined) { + // ELSE clause exists. + state = CaseState.Body + curr = Some(elseBody.get) + } else { + // No remaining clauses. + curr = None + } + } + condition + case CaseState.Body => + assert(curr.get.isInstanceOf[CompoundBodyExec]) + val currBody = curr.get.asInstanceOf[CompoundBodyExec] + val retStmt = currBody.getTreeIterator.next() + if (!currBody.getTreeIterator.hasNext) { + curr = None + } + retStmt + } + } + + override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator + + override def reset(): Unit = { + state = CaseState.Condition + curr = Some(conditions.head) + clauseIdx = 0 + conditions.foreach(c => c.reset()) + conditionalBodies.foreach(b => b.reset()) + elseBody.foreach(b => b.reset()) + } +} + /** * Executable node for RepeatStatement. * @param condition Executable node for the condition - evaluates to a row with a single boolean diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala index 865b33999655a..917b4d6f45ee0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.scripting import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier -import org.apache.spark.sql.catalyst.parser.{CompoundBody, CompoundPlanStatement, IfElseStatement, IterateStatement, LeaveStatement, RepeatStatement, SingleStatement, WhileStatement} +import org.apache.spark.sql.catalyst.parser.{CaseStatement, CompoundBody, CompoundPlanStatement, IfElseStatement, IterateStatement, LeaveStatement, RepeatStatement, SingleStatement, WhileStatement} import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, DropVariable, LogicalPlan} import org.apache.spark.sql.catalyst.trees.Origin @@ -95,6 +95,17 @@ case class SqlScriptingInterpreter() { new IfElseStatementExec( conditionsExec, conditionalBodiesExec, unconditionalBodiesExec, session) + case CaseStatement(conditions, conditionalBodies, elseBody) => + val conditionsExec = conditions.map(condition => + // todo: what to put here for isInternal, in case of simple case statement + new SingleStatementExec(condition.parsedPlan, condition.origin, isInternal = false)) + val conditionalBodiesExec = conditionalBodies.map(body => + transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec]) + val unconditionalBodiesExec = elseBody.map(body => + transformTreeIntoExecutable(body, session).asInstanceOf[CompoundBodyExec]) + new CaseStatementExec( + conditionsExec, conditionalBodiesExec, unconditionalBodiesExec, session) + case WhileStatement(condition, body, label) => val conditionExec = new SingleStatementExec(condition.parsedPlan, condition.origin, isInternal = false) 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 14ac67eb93a32..83c9ebfef4b25 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 @@ -444,21 +444,1493 @@ DropTable false, false -- !query -create table t5(str string collate utf8_binary, delimiter string collate utf8_lcase, partNum int) using parquet +create table t5(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t5`, false -- !query -insert into t5 values('11AB12AB13', 'AB', 2) +insert into t5 values ('Spark', 'Spark', 'SQL') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [str, delimiter, partNum] -+- Project [cast(col1#x as string) AS str#x, cast(col2#x as string collate UTF8_LCASE) AS delimiter#x, cast(col3#x as int) AS partNum#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x, col3#x] -- !query -select split_part(str, delimiter, partNum) from t5 +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('bbAbaAbA', 'bbAbAAbA', 'a') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('İo', 'İo', 'İo') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('İo', 'İo', 'i̇o') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('efd2', 'efd2', 'efd2') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('kitten', 'kitten', 'sitTing') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('abc', 'abc', 'abc') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +insert into t5 values ('abcdcba', 'abcdcba', 'aBcDCbA') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t5, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t5], Append, `spark_catalog`.`default`.`t5`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t5), [s, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS s#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +create table t6(ascii long) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t6`, false + + +-- !query +insert into t6 values (97) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t6, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t6], Append, `spark_catalog`.`default`.`t6`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t6), [ascii] ++- Project [cast(col1#x as bigint) AS ascii#xL] + +- LocalRelation [col1#x] + + +-- !query +insert into t6 values (66) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t6, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t6], Append, `spark_catalog`.`default`.`t6`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t6), [ascii] ++- Project [cast(col1#x as bigint) AS ascii#xL] + +- LocalRelation [col1#x] + + +-- !query +create table t7(ascii double) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t7`, false + + +-- !query +insert into t7 values (97.52143) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t7, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t7], Append, `spark_catalog`.`default`.`t7`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t7), [ascii] ++- Project [cast(col1#x as double) AS ascii#x] + +- LocalRelation [col1#x] + + +-- !query +insert into t7 values (66.421) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t7, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t7], Append, `spark_catalog`.`default`.`t7`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t7), [ascii] ++- Project [cast(col1#x as double) AS ascii#x] + +- LocalRelation [col1#x] + + +-- !query +create table t8(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t8`, false + + +-- !query +insert into t8 values ('%s%s', 'abCdE', 'abCdE') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t8, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t8], Append, `spark_catalog`.`default`.`t8`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t8), [format, utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS format#x, cast(col2#x as string) AS utf8_binary#x, cast(col3#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +create table t9(num long) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t9`, false + + +-- !query +insert into t9 values (97) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t9, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t9], Append, `spark_catalog`.`default`.`t9`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t9), [num] ++- Project [cast(col1#x as bigint) AS num#xL] + +- LocalRelation [col1#x] + + +-- !query +insert into t9 values (66) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t9, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t9], Append, `spark_catalog`.`default`.`t9`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t9), [num] ++- Project [cast(col1#x as bigint) AS num#xL] + +- LocalRelation [col1#x] + + +-- !query +create table t10(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t10`, false + + +-- !query +insert into t10 values ('aaAaAAaA', 'aaAaaAaA') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t10], Append, `spark_catalog`.`default`.`t10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t10), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +insert into t10 values ('efd2', 'efd2') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t10, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t10], Append, `spark_catalog`.`default`.`t10`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t10), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +-- !query analysis +Project [concat_ws(cast( as string collate UTF8_LCASE), utf8_lcase#x, utf8_lcase#x) AS concat_ws( , utf8_lcase, utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select concat_ws(' ', utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t5 +-- !query analysis +Project [concat_ws(collate( , utf8_lcase), cast(utf8_binary#x as string collate UTF8_LCASE), collate(SQL, utf8_lcase)) AS concat_ws(collate( , utf8_lcase), utf8_binary, collate(SQL, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5 +-- !query analysis +Project [concat_ws(cast(, as string collate UTF8_LCASE), utf8_lcase#x, cast(word as string collate UTF8_LCASE)) AS concat_ws(,, utf8_lcase, word)#x, concat_ws(,, utf8_binary#x, word) AS concat_ws(,, utf8_binary, word)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5 +-- !query analysis +Project [concat_ws(,, cast(utf8_lcase#x as string), collate(word, utf8_binary)) AS concat_ws(,, utf8_lcase, collate(word, utf8_binary))#x, concat_ws(cast(, as string collate UTF8_LCASE), cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase)) AS concat_ws(,, utf8_binary, collate(word, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select elt(2, s, utf8_binary) from t5 +-- !query analysis +Project [elt(2, s#x, utf8_binary#x, false) AS elt(2, s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select elt(2, utf8_binary, utf8_lcase, s) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [elt(1, collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), false) AS elt(1, collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5 +-- !query analysis +Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string), false) AS elt(1, collate(utf8_binary, utf8_binary), utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5 +-- !query analysis +Project [elt(1, utf8_binary#x, word, false) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, cast(word as string collate UTF8_LCASE), false) AS elt(1, utf8_lcase, word)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5 +-- !query analysis +Project [elt(1, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase), false) AS elt(1, utf8_binary, collate(word, utf8_lcase))#x, elt(1, cast(utf8_lcase#x as string), collate(word, utf8_binary), false) AS elt(1, utf8_lcase, collate(word, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select split_part(utf8_binary, utf8_lcase, 3) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select split_part(s, utf8_binary, 1) from t5 +-- !query analysis +Project [split_part(s#x, utf8_binary#x, 1) AS split_part(s, utf8_binary, 1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query analysis +Project [split_part(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2) AS split_part(utf8_binary, collate(utf8_lcase, utf8_binary), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query analysis +Project [split_part(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2) AS split_part(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5 +-- !query analysis +Project [split_part(utf8_binary#x, a, 3) AS split_part(utf8_binary, a, 3)#x, split_part(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 3) AS split_part(utf8_lcase, a, 3)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5 +-- !query analysis +Project [split_part(cast(utf8_binary#x as string collate UTF8_LCASE), collate(a, utf8_lcase), 3) AS split_part(utf8_binary, collate(a, utf8_lcase), 3)#x, split_part(cast(utf8_lcase#x as string), collate(a, utf8_binary), 3) AS split_part(utf8_lcase, collate(a, utf8_binary), 3)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select contains(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select contains(s, utf8_binary) from t5 +-- !query analysis +Project [Contains(s#x, utf8_binary#x) AS contains(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select contains(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [Contains(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS contains(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [Contains(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS contains(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5 +-- !query analysis +Project [Contains(utf8_binary#x, a) AS contains(utf8_binary, a)#x, Contains(utf8_lcase#x, cast(a as string collate UTF8_LCASE)) AS contains(utf8_lcase, a)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t5 +-- !query analysis +Project [Contains(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase)) AS contains(utf8_binary, collate(AaAA, utf8_lcase))#x, Contains(cast(utf8_lcase#x as string), collate(AAa, utf8_binary)) AS contains(utf8_lcase, collate(AAa, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substring_index(utf8_binary, utf8_lcase, 2) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select substring_index(s, utf8_binary,1) from t5 +-- !query analysis +Project [substring_index(s#x, utf8_binary#x, 1) AS substring_index(s, utf8_binary, 1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query analysis +Project [substring_index(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2) AS substring_index(utf8_binary, collate(utf8_lcase, utf8_binary), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query analysis +Project [substring_index(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2) AS substring_index(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5 +-- !query analysis +Project [substring_index(utf8_binary#x, a, 2) AS substring_index(utf8_binary, a, 2)#x, substring_index(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 2) AS substring_index(utf8_lcase, a, 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query analysis +Project [substring_index(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 2) AS substring_index(utf8_binary, collate(AaAA, utf8_lcase), 2)#x, substring_index(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2) AS substring_index(utf8_lcase, collate(AAa, utf8_binary), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select instr(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select instr(s, utf8_binary) from t5 +-- !query analysis +Project [instr(s#x, utf8_binary#x) AS instr(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select instr(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [instr(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS instr(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [instr(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS instr(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5 +-- !query analysis +Project [instr(utf8_binary#x, a) AS instr(utf8_binary, a)#x, instr(utf8_lcase#x, cast(a as string collate UTF8_LCASE)) AS instr(utf8_lcase, a)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t5 +-- !query analysis +Project [instr(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase)) AS instr(utf8_binary, collate(AaAA, utf8_lcase))#x, instr(cast(utf8_lcase#x as string), collate(AAa, utf8_binary)) AS instr(utf8_lcase, collate(AAa, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select find_in_set(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select find_in_set(s, utf8_binary) from t5 +-- !query analysis +Project [find_in_set(s#x, utf8_binary#x) AS find_in_set(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [find_in_set(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS find_in_set(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [find_in_set(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS find_in_set(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5 +-- !query analysis +Project [find_in_set(utf8_binary#x, aaAaaAaA,i̇o) AS find_in_set(utf8_binary, aaAaaAaA,i̇o)#x, find_in_set(utf8_lcase#x, cast(aaAaaAaA,i̇o as string collate UTF8_LCASE)) AS find_in_set(utf8_lcase, aaAaaAaA,i̇o)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5 +-- !query analysis +Project [find_in_set(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA,i̇o, utf8_lcase)) AS find_in_set(utf8_binary, collate(aaAaaAaA,i̇o, utf8_lcase))#x, find_in_set(cast(utf8_lcase#x as string), collate(aaAaaAaA,i̇o, utf8_binary)) AS find_in_set(utf8_lcase, collate(aaAaaAaA,i̇o, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select startswith(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select startswith(s, utf8_binary) from t5 +-- !query analysis +Project [StartsWith(s#x, utf8_binary#x) AS startswith(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [StartsWith(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS startswith(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [StartsWith(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS startswith(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5 +-- !query analysis +Project [StartsWith(utf8_binary#x, aaAaaAaA) AS startswith(utf8_binary, aaAaaAaA)#x, StartsWith(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE)) AS startswith(utf8_lcase, aaAaaAaA)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query analysis +Project [StartsWith(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase)) AS startswith(utf8_binary, collate(aaAaaAaA, utf8_lcase))#x, StartsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS startswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select translate(utf8_lcase, utf8_lcase, '12345') from t5 +-- !query analysis +Project [translate(utf8_lcase#x, utf8_lcase#x, cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_lcase, utf8_lcase, 12345)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select translate(utf8_binary, utf8_lcase, '12345') from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" + } +} + + +-- !query +select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t5 +-- !query analysis +Project [translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(SQL, utf8_lcase), collate(12345, utf8_lcase)) AS translate(utf8_binary, collate(SQL, utf8_lcase), collate(12345, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5 +-- !query analysis +Project [translate(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE), cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_lcase, aaAaaAaA, 12345)#x, translate(utf8_binary#x, aaAaaAaA, 12345) AS translate(utf8_binary, aaAaaAaA, 12345)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5 +-- !query analysis +Project [translate(cast(utf8_lcase#x as string), collate(aBc, utf8_binary), 12345) AS translate(utf8_lcase, collate(aBc, utf8_binary), 12345)#x, translate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aBc, utf8_lcase), cast(12345 as string collate UTF8_LCASE)) AS translate(utf8_binary, collate(aBc, utf8_lcase), 12345)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select replace(utf8_binary, utf8_lcase, 'abc') from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select replace(s, utf8_binary, 'abc') from t5 +-- !query analysis +Project [replace(s#x, utf8_binary#x, abc) AS replace(s, utf8_binary, abc)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t5 +-- !query analysis +Project [replace(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), abc) AS replace(utf8_binary, collate(utf8_lcase, utf8_binary), abc)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5 +-- !query analysis +Project [replace(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), cast(abc as string collate UTF8_LCASE)) AS replace(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), abc)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5 +-- !query analysis +Project [replace(utf8_binary#x, aaAaaAaA, abc) AS replace(utf8_binary, aaAaaAaA, abc)#x, replace(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE), cast(abc as string collate UTF8_LCASE)) AS replace(utf8_lcase, aaAaaAaA, abc)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5 +-- !query analysis +Project [replace(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase), cast(abc as string collate UTF8_LCASE)) AS replace(utf8_binary, collate(aaAaaAaA, utf8_lcase), abc)#x, replace(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary), abc) AS replace(utf8_lcase, collate(aaAaaAaA, utf8_binary), abc)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select endswith(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select endswith(s, utf8_binary) from t5 +-- !query analysis +Project [EndsWith(s#x, utf8_binary#x) AS endswith(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [EndsWith(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS endswith(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [EndsWith(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS endswith(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5 +-- !query analysis +Project [EndsWith(utf8_binary#x, aaAaaAaA) AS endswith(utf8_binary, aaAaaAaA)#x, EndsWith(utf8_lcase#x, cast(aaAaaAaA as string collate UTF8_LCASE)) AS endswith(utf8_lcase, aaAaaAaA)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query analysis +Project [EndsWith(cast(utf8_binary#x as string collate UTF8_LCASE), collate(aaAaaAaA, utf8_lcase)) AS endswith(utf8_binary, collate(aaAaaAaA, utf8_lcase))#x, EndsWith(cast(utf8_lcase#x as string), collate(aaAaaAaA, utf8_binary)) AS endswith(utf8_lcase, collate(aaAaaAaA, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t5 +-- !query analysis +Project [repeat(utf8_binary#x, 3) AS repeat(utf8_binary, 3)#x, repeat(utf8_lcase#x, 2) AS repeat(utf8_lcase, 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t5 +-- !query analysis +Project [repeat(collate(utf8_binary#x, utf8_lcase), 3) AS repeat(collate(utf8_binary, utf8_lcase), 3)#x, repeat(collate(utf8_lcase#x, utf8_binary), 2) AS repeat(collate(utf8_lcase, utf8_binary), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select ascii(utf8_binary), ascii(utf8_lcase) from t5 +-- !query analysis +Project [ascii(utf8_binary#x) AS ascii(utf8_binary)#x, ascii(utf8_lcase#x) AS ascii(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [ascii(collate(utf8_binary#x, utf8_lcase)) AS ascii(collate(utf8_binary, utf8_lcase))#x, ascii(collate(utf8_lcase#x, utf8_binary)) AS ascii(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select unbase64(utf8_binary), unbase64(utf8_lcase) from t10 +-- !query analysis +Project [unbase64(utf8_binary#x, false) AS unbase64(utf8_binary)#x, unbase64(utf8_lcase#x, false) AS unbase64(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t10 + +- Relation spark_catalog.default.t10[utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t10 +-- !query analysis +Project [unbase64(collate(utf8_binary#x, utf8_lcase), false) AS unbase64(collate(utf8_binary, utf8_lcase))#x, unbase64(collate(utf8_lcase#x, utf8_binary), false) AS unbase64(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t10 + +- Relation spark_catalog.default.t10[utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select chr(ascii) from t6 +-- !query analysis +Project [chr(ascii#xL) AS chr(ascii)#x] ++- SubqueryAlias spark_catalog.default.t6 + +- Relation spark_catalog.default.t6[ascii#xL] parquet + + +-- !query +select base64(utf8_binary), base64(utf8_lcase) from t5 +-- !query analysis +Project [base64(cast(utf8_binary#x as binary)) AS base64(utf8_binary)#x, base64(cast(utf8_lcase#x as binary)) AS base64(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [base64(cast(collate(utf8_binary#x, utf8_lcase) as binary)) AS base64(collate(utf8_binary, utf8_lcase))#x, base64(cast(collate(utf8_lcase#x, utf8_binary) as binary)) AS base64(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t5 +-- !query analysis +Project [decode(encode(utf8_binary#x, utf-8), utf-8) AS decode(encode(utf8_binary, utf-8), utf-8)#x, decode(encode(utf8_lcase#x, utf-8), utf-8) AS decode(encode(utf8_lcase, utf-8), utf-8)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t5 +-- !query analysis +Project [decode(encode(collate(utf8_binary#x, utf8_lcase), utf-8), utf-8) AS decode(encode(collate(utf8_binary, utf8_lcase), utf-8), utf-8)#x, decode(encode(collate(utf8_lcase#x, utf8_binary), utf-8), utf-8) AS decode(encode(collate(utf8_lcase, utf8_binary), utf-8), utf-8)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select format_number(ascii, '###.###') from t7 +-- !query analysis +Project [format_number(ascii#x, ###.###) AS format_number(ascii, ###.###)#x] ++- SubqueryAlias spark_catalog.default.t7 + +- Relation spark_catalog.default.t7[ascii#x] parquet + + +-- !query +select format_number(ascii, '###.###' collate utf8_lcase) from t7 +-- !query analysis +Project [format_number(ascii#x, collate(###.###, utf8_lcase)) AS format_number(ascii, collate(###.###, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t7 + +- Relation spark_catalog.default.t7[ascii#x] parquet + + +-- !query +select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t5 +-- !query analysis +Project [encode(utf8_binary#x, utf-8) AS encode(utf8_binary, utf-8)#x, encode(utf8_lcase#x, utf-8) AS encode(utf8_lcase, utf-8)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t5 +-- !query analysis +Project [encode(collate(utf8_binary#x, utf8_lcase), utf-8) AS encode(collate(utf8_binary, utf8_lcase), utf-8)#x, encode(collate(utf8_lcase#x, utf8_binary), utf-8) AS encode(collate(utf8_lcase, utf8_binary), utf-8)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t5 +-- !query analysis +Project [to_binary(utf8_binary#x, Some(utf-8), false) AS to_binary(utf8_binary, utf-8)#x, to_binary(utf8_lcase#x, Some(utf-8), false) AS to_binary(utf8_lcase, utf-8)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t5 +-- !query analysis +Project [to_binary(collate(utf8_binary#x, utf8_lcase), Some(utf-8), false) AS to_binary(collate(utf8_binary, utf8_lcase), utf-8)#x, to_binary(collate(utf8_lcase#x, utf8_binary), Some(utf-8), false) AS to_binary(collate(utf8_lcase, utf8_binary), utf-8)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select sentences(utf8_binary), sentences(utf8_lcase) from t5 +-- !query analysis +Project [sentences(utf8_binary#x, , ) AS sentences(utf8_binary, , )#x, sentences(utf8_lcase#x, , ) AS sentences(utf8_lcase, , )#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [sentences(collate(utf8_binary#x, utf8_lcase), , ) AS sentences(collate(utf8_binary, utf8_lcase), , )#x, sentences(collate(utf8_lcase#x, utf8_binary), , ) AS sentences(collate(utf8_lcase, utf8_binary), , )#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select upper(utf8_binary), upper(utf8_lcase) from t5 +-- !query analysis +Project [upper(utf8_binary#x) AS upper(utf8_binary)#x, upper(utf8_lcase#x) AS upper(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [upper(collate(utf8_binary#x, utf8_lcase)) AS upper(collate(utf8_binary, utf8_lcase))#x, upper(collate(utf8_lcase#x, utf8_binary)) AS upper(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select lower(utf8_binary), lower(utf8_lcase) from t5 +-- !query analysis +Project [lower(utf8_binary#x) AS lower(utf8_binary)#x, lower(utf8_lcase#x) AS lower(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [lower(collate(utf8_binary#x, utf8_lcase)) AS lower(collate(utf8_binary, utf8_lcase))#x, lower(collate(utf8_lcase#x, utf8_binary)) AS lower(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select initcap(utf8_binary), initcap(utf8_lcase) from t5 +-- !query analysis +Project [initcap(utf8_binary#x) AS initcap(utf8_binary)#x, initcap(utf8_lcase#x) AS initcap(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [initcap(collate(utf8_binary#x, utf8_lcase)) AS initcap(collate(utf8_binary, utf8_lcase))#x, initcap(collate(utf8_lcase#x, utf8_binary)) AS initcap(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select overlay(utf8_binary, utf8_lcase, 2) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select overlay(s, utf8_binary,1) from t5 +-- !query analysis +Project [overlay(s#x, utf8_binary#x, 1, -1) AS overlay(s, utf8_binary, 1, -1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query analysis +Project [overlay(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 2, -1) AS overlay(utf8_binary, collate(utf8_lcase, utf8_binary), 2, -1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query analysis +Project [overlay(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 2, -1) AS overlay(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 2, -1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5 +-- !query analysis +Project [overlay(utf8_binary#x, a, 2, -1) AS overlay(utf8_binary, a, 2, -1)#x, overlay(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 2, -1) AS overlay(utf8_lcase, a, 2, -1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query analysis +Project [overlay(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 2, -1) AS overlay(utf8_binary, collate(AaAA, utf8_lcase), 2, -1)#x, overlay(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 2, -1) AS overlay(utf8_lcase, collate(AAa, utf8_binary), 2, -1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select format_string(format, utf8_binary, utf8_lcase) from t8 +-- !query analysis +Project [format_string(format#x, utf8_binary#x, utf8_lcase#x) AS format_string(format, utf8_binary, utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t8 + +- Relation spark_catalog.default.t8[format#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t8 +-- !query analysis +Project [format_string(collate(format#x, utf8_lcase), utf8_lcase#x, collate(utf8_binary#x, utf8_lcase), 3) AS format_string(collate(format, utf8_lcase), utf8_lcase, collate(utf8_binary, utf8_lcase), 3)#x, format_string(format#x, collate(utf8_lcase#x, utf8_binary), utf8_binary#x) AS format_string(format, collate(utf8_lcase, utf8_binary), utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t8 + +- Relation spark_catalog.default.t8[format#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select format_string(format, utf8_binary, utf8_lcase) from t8 +-- !query analysis +Project [format_string(format#x, utf8_binary#x, utf8_lcase#x) AS format_string(format, utf8_binary, utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t8 + +- Relation spark_catalog.default.t8[format#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select soundex(utf8_binary), soundex(utf8_lcase) from t5 +-- !query analysis +Project [soundex(utf8_binary#x) AS soundex(utf8_binary)#x, soundex(utf8_lcase#x) AS soundex(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [soundex(collate(utf8_binary#x, utf8_lcase)) AS soundex(collate(utf8_binary, utf8_lcase))#x, soundex(collate(utf8_lcase#x, utf8_binary)) AS soundex(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select length(utf8_binary), length(utf8_lcase) from t5 +-- !query analysis +Project [length(utf8_binary#x) AS length(utf8_binary)#x, length(utf8_lcase#x) AS length(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [length(collate(utf8_binary#x, utf8_lcase)) AS length(collate(utf8_binary, utf8_lcase))#x, length(collate(utf8_lcase#x, utf8_binary)) AS length(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select bit_length(utf8_binary), bit_length(utf8_lcase) from t5 +-- !query analysis +Project [bit_length(utf8_binary#x) AS bit_length(utf8_binary)#x, bit_length(utf8_lcase#x) AS bit_length(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [bit_length(collate(utf8_binary#x, utf8_lcase)) AS bit_length(collate(utf8_binary, utf8_lcase))#x, bit_length(collate(utf8_lcase#x, utf8_binary)) AS bit_length(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select octet_length(utf8_binary), octet_length(utf8_lcase) from t5 +-- !query analysis +Project [octet_length(utf8_binary#x) AS octet_length(utf8_binary)#x, octet_length(utf8_lcase#x) AS octet_length(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [octet_length(collate(utf8_binary#x, utf8_lcase)) AS octet_length(collate(utf8_binary, utf8_lcase))#x, octet_length(collate(utf8_lcase#x, utf8_binary)) AS octet_length(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select luhn_check(num) from t9 +-- !query analysis +Project [luhn_check(cast(num#xL as string)) AS luhn_check(num)#x] ++- SubqueryAlias spark_catalog.default.t9 + +- Relation spark_catalog.default.t9[num#xL] parquet + + +-- !query +select levenshtein(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select levenshtein(s, utf8_binary) from t5 +-- !query analysis +Project [levenshtein(s#x, utf8_binary#x, None) AS levenshtein(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [levenshtein(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), None) AS levenshtein(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [levenshtein(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), None) AS levenshtein(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5 +-- !query analysis +Project [levenshtein(utf8_binary#x, a, None) AS levenshtein(utf8_binary, a)#x, levenshtein(utf8_lcase#x, cast(a as string collate UTF8_LCASE), None) AS levenshtein(utf8_lcase, a)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5 +-- !query analysis +Project [levenshtein(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), Some(3)) AS levenshtein(utf8_binary, collate(AaAA, utf8_lcase), 3)#x, levenshtein(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), Some(4)) AS levenshtein(utf8_lcase, collate(AAa, utf8_binary), 4)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5 +-- !query analysis +Project [is_valid_utf8(utf8_binary#x) AS is_valid_utf8(utf8_binary)#x, is_valid_utf8(utf8_lcase#x) AS is_valid_utf8(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [is_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS is_valid_utf8(collate(utf8_binary, utf8_lcase))#x, is_valid_utf8(collate(utf8_lcase#x, utf8_binary)) AS is_valid_utf8(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5 +-- !query analysis +Project [make_valid_utf8(utf8_binary#x) AS make_valid_utf8(utf8_binary)#x, make_valid_utf8(utf8_lcase#x) AS make_valid_utf8(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [make_valid_utf8(collate(utf8_binary#x, utf8_lcase)) AS make_valid_utf8(collate(utf8_binary, utf8_lcase))#x, make_valid_utf8(collate(utf8_lcase#x, utf8_binary)) AS make_valid_utf8(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5 +-- !query analysis +Project [validate_utf8(utf8_binary#x) AS validate_utf8(utf8_binary)#x, validate_utf8(utf8_lcase#x) AS validate_utf8(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS validate_utf8(collate(utf8_binary, utf8_lcase))#x, validate_utf8(collate(utf8_lcase#x, utf8_binary)) AS validate_utf8(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5 +-- !query analysis +Project [try_validate_utf8(utf8_binary#x) AS try_validate_utf8(utf8_binary)#x, try_validate_utf8(utf8_lcase#x) AS try_validate_utf8(utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [try_validate_utf8(collate(utf8_binary#x, utf8_lcase)) AS try_validate_utf8(collate(utf8_binary, utf8_lcase))#x, try_validate_utf8(collate(utf8_lcase#x, utf8_binary)) AS try_validate_utf8(collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5 +-- !query analysis +Project [substr(utf8_binary#x, 2, 2) AS substr(utf8_binary, 2, 2)#x, substr(utf8_lcase#x, 2, 2) AS substr(utf8_lcase, 2, 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t5 +-- !query analysis +Project [substr(collate(utf8_binary#x, utf8_lcase), 2, 2) AS substr(collate(utf8_binary, utf8_lcase), 2, 2)#x, substr(collate(utf8_lcase#x, utf8_binary), 2, 2) AS substr(collate(utf8_lcase, utf8_binary), 2, 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select right(utf8_binary, 2), right(utf8_lcase, 2) from t5 +-- !query analysis +Project [right(utf8_binary#x, 2) AS right(utf8_binary, 2)#x, right(utf8_lcase#x, 2) AS right(utf8_lcase, 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t5 +-- !query analysis +Project [right(collate(utf8_binary#x, utf8_lcase), 2) AS right(collate(utf8_binary, utf8_lcase), 2)#x, right(collate(utf8_lcase#x, utf8_binary), 2) AS right(collate(utf8_lcase, utf8_binary), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t5 +-- !query analysis +Project [left(utf8_binary#x, cast(collate(2, utf8_lcase) as int)) AS left(utf8_binary, collate(2, utf8_lcase))#x, left(utf8_lcase#x, 2) AS left(utf8_lcase, 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t5 +-- !query analysis +Project [left(collate(utf8_binary#x, utf8_lcase), 2) AS left(collate(utf8_binary, utf8_lcase), 2)#x, left(collate(utf8_lcase#x, utf8_binary), 2) AS left(collate(utf8_lcase, utf8_binary), 2)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select rpad(utf8_binary, 8, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select rpad(s, 8, utf8_binary) from t5 +-- !query analysis +Project [rpad(s#x, 8, utf8_binary#x) AS rpad(s, 8, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [rpad(utf8_binary#x, 8, collate(utf8_lcase#x, utf8_binary)) AS rpad(utf8_binary, 8, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [rpad(collate(utf8_binary#x, utf8_lcase), 8, collate(utf8_lcase#x, utf8_lcase)) AS rpad(collate(utf8_binary, utf8_lcase), 8, collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5 +-- !query analysis +Project [rpad(utf8_binary#x, 8, a) AS rpad(utf8_binary, 8, a)#x, rpad(utf8_lcase#x, 8, cast(a as string collate UTF8_LCASE)) AS rpad(utf8_lcase, 8, a)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5 +-- !query analysis +Project [rpad(cast(utf8_binary#x as string collate UTF8_LCASE), 8, collate(AaAA, utf8_lcase)) AS rpad(utf8_binary, 8, collate(AaAA, utf8_lcase))#x, rpad(cast(utf8_lcase#x as string), 8, collate(AAa, utf8_binary)) AS rpad(utf8_lcase, 8, collate(AAa, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select lpad(utf8_binary, 8, utf8_lcase) from t5 -- !query analysis org.apache.spark.sql.AnalysisException { @@ -468,7 +1940,15 @@ org.apache.spark.sql.AnalysisException -- !query -select split_part(str collate utf8_binary, delimiter collate utf8_lcase, partNum) from t5 +select lpad(s, 8, utf8_binary) from t5 +-- !query analysis +Project [lpad(s#x, 8, utf8_binary#x) AS lpad(s, 8, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5 -- !query analysis org.apache.spark.sql.AnalysisException { @@ -481,36 +1961,39 @@ org.apache.spark.sql.AnalysisException -- !query -select split_part(str collate utf8_binary, delimiter collate utf8_binary, partNum) from t5 +select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5 -- !query analysis -Project [split_part(collate(str#x, utf8_binary), collate(delimiter#x, utf8_binary), partNum#x) AS split_part(collate(str, utf8_binary), collate(delimiter, utf8_binary), partNum)#x] +Project [lpad(utf8_binary#x, 8, collate(utf8_lcase#x, utf8_binary)) AS lpad(utf8_binary, 8, collate(utf8_lcase, utf8_binary))#x] +- SubqueryAlias spark_catalog.default.t5 - +- Relation spark_catalog.default.t5[str#x,delimiter#x,partNum#x] parquet + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet -- !query -drop table t5 +select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5 -- !query analysis -DropTable false, false -+- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t5 +Project [lpad(collate(utf8_binary#x, utf8_lcase), 8, collate(utf8_lcase#x, utf8_lcase)) AS lpad(collate(utf8_binary, utf8_lcase), 8, collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet -- !query -create table t6 (utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase, threshold int) using parquet +select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5 -- !query analysis -CreateDataSourceTableCommand `spark_catalog`.`default`.`t6`, false +Project [lpad(utf8_binary#x, 8, a) AS lpad(utf8_binary, 8, a)#x, lpad(utf8_lcase#x, 8, cast(a as string collate UTF8_LCASE)) AS lpad(utf8_lcase, 8, a)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet -- !query -insert into t6 values('kitten', 'sitting', 2) +select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5 -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t6, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t6], Append, `spark_catalog`.`default`.`t6`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t6), [utf8_binary, utf8_lcase, threshold] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x, cast(col3#x as int) AS threshold#x] - +- LocalRelation [col1#x, col2#x, col3#x] +Project [lpad(cast(utf8_binary#x as string collate UTF8_LCASE), 8, collate(AaAA, utf8_lcase)) AS lpad(utf8_binary, 8, collate(AaAA, utf8_lcase))#x, lpad(cast(utf8_lcase#x as string), 8, collate(AAa, utf8_binary)) AS lpad(utf8_lcase, 8, collate(AAa, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet -- !query -select levenshtein(utf8_binary, utf8_lcase) from t6 +select locate(utf8_binary, utf8_lcase) from t5 -- !query analysis org.apache.spark.sql.AnalysisException { @@ -520,7 +2003,15 @@ org.apache.spark.sql.AnalysisException -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t6 +select locate(s, utf8_binary) from t5 +-- !query analysis +Project [locate(s#x, utf8_binary#x, 1) AS locate(s, utf8_binary, 1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 -- !query analysis org.apache.spark.sql.AnalysisException { @@ -533,15 +2024,102 @@ org.apache.spark.sql.AnalysisException -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t6 +select locate(utf8_binary, utf8_lcase collate utf8_binary) from t5 -- !query analysis -Project [levenshtein(collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), None) AS levenshtein(collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary))#x] -+- SubqueryAlias spark_catalog.default.t6 - +- Relation spark_catalog.default.t6[utf8_binary#x,utf8_lcase#x,threshold#x] parquet +Project [locate(utf8_binary#x, collate(utf8_lcase#x, utf8_binary), 1) AS locate(utf8_binary, collate(utf8_lcase, utf8_binary), 1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t5 +-- !query analysis +Project [locate(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase), 3) AS locate(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase), 3)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5 +-- !query analysis +Project [locate(utf8_binary#x, a, 1) AS locate(utf8_binary, a, 1)#x, locate(utf8_lcase#x, cast(a as string collate UTF8_LCASE), 1) AS locate(utf8_lcase, a, 1)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5 +-- !query analysis +Project [locate(cast(utf8_binary#x as string collate UTF8_LCASE), collate(AaAA, utf8_lcase), 4) AS locate(utf8_binary, collate(AaAA, utf8_lcase), 4)#x, locate(cast(utf8_lcase#x as string), collate(AAa, utf8_binary), 4) AS locate(utf8_lcase, collate(AAa, utf8_binary), 4)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select TRIM(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select TRIM(s, utf8_binary) from t5 +-- !query analysis +Project [trim(utf8_binary#x, Some(s#x)) AS TRIM(BOTH s FROM utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" + } +} + + +-- !query +select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [trim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(BOTH utf8_binary FROM collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [trim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(BOTH collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5 +-- !query analysis +Project [trim(utf8_binary#x, Some(ABc)) AS TRIM(BOTH ABc FROM utf8_binary)#x, trim(utf8_lcase#x, Some(cast(ABc as string collate UTF8_LCASE))) AS TRIM(BOTH ABc FROM utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet -- !query -select levenshtein(utf8_binary, utf8_lcase, threshold) from t6 +select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query analysis +Project [trim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(BOTH collate(ABc, utf8_lcase) FROM utf8_binary)#x, trim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(BOTH collate(AAa, utf8_binary) FROM utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select BTRIM(utf8_binary, utf8_lcase) from t5 -- !query analysis org.apache.spark.sql.AnalysisException { @@ -551,7 +2129,15 @@ org.apache.spark.sql.AnalysisException -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase, threshold) from t6 +select BTRIM(s, utf8_binary) from t5 +-- !query analysis +Project [btrim(s#x, utf8_binary#x) AS btrim(s, utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 -- !query analysis org.apache.spark.sql.AnalysisException { @@ -564,11 +2150,168 @@ org.apache.spark.sql.AnalysisException -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary, threshold) from t6 +select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 -- !query analysis -Project [levenshtein(collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), Some(threshold#x)) AS levenshtein(collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary), threshold)#x] -+- SubqueryAlias spark_catalog.default.t6 - +- Relation spark_catalog.default.t6[utf8_binary#x,utf8_lcase#x,threshold#x] parquet +Project [btrim(utf8_binary#x, collate(utf8_lcase#x, utf8_binary)) AS btrim(utf8_binary, collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [btrim(collate(utf8_binary#x, utf8_lcase), collate(utf8_lcase#x, utf8_lcase)) AS btrim(collate(utf8_binary, utf8_lcase), collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t5 +-- !query analysis +Project [btrim(ABc, utf8_binary#x) AS btrim(ABc, utf8_binary)#x, btrim(ABc, utf8_lcase#x) AS btrim(ABc, utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query analysis +Project [btrim(collate(ABc, utf8_lcase), utf8_binary#x) AS btrim(collate(ABc, utf8_lcase), utf8_binary)#x, btrim(collate(AAa, utf8_binary), utf8_lcase#x) AS btrim(collate(AAa, utf8_binary), utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select LTRIM(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select LTRIM(s, utf8_binary) from t5 +-- !query analysis +Project [ltrim(utf8_binary#x, Some(s#x)) AS TRIM(LEADING s FROM utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" + } +} + + +-- !query +select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [ltrim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(LEADING utf8_binary FROM collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [ltrim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(LEADING collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5 +-- !query analysis +Project [ltrim(utf8_binary#x, Some(ABc)) AS TRIM(LEADING ABc FROM utf8_binary)#x, ltrim(utf8_lcase#x, Some(cast(ABc as string collate UTF8_LCASE))) AS TRIM(LEADING ABc FROM utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query analysis +Project [ltrim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(LEADING collate(ABc, utf8_lcase) FROM utf8_binary)#x, ltrim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(LEADING collate(AAa, utf8_binary) FROM utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select RTRIM(utf8_binary, utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select RTRIM(s, utf8_binary) from t5 +-- !query analysis +Project [rtrim(utf8_binary#x, Some(s#x)) AS TRIM(TRAILING s FROM utf8_binary)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" + } +} + + +-- !query +select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query analysis +Project [rtrim(collate(utf8_lcase#x, utf8_binary), Some(utf8_binary#x)) AS TRIM(TRAILING utf8_binary FROM collate(utf8_lcase, utf8_binary))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query analysis +Project [rtrim(collate(utf8_lcase#x, utf8_lcase), Some(collate(utf8_binary#x, utf8_lcase))) AS TRIM(TRAILING collate(utf8_binary, utf8_lcase) FROM collate(utf8_lcase, utf8_lcase))#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5 +-- !query analysis +Project [rtrim(utf8_binary#x, Some(ABc)) AS TRIM(TRAILING ABc FROM utf8_binary)#x, rtrim(utf8_lcase#x, Some(cast(ABc as string collate UTF8_LCASE))) AS TRIM(TRAILING ABc FROM utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query analysis +Project [rtrim(cast(utf8_binary#x as string collate UTF8_LCASE), Some(collate(ABc, utf8_lcase))) AS TRIM(TRAILING collate(ABc, utf8_lcase) FROM utf8_binary)#x, rtrim(cast(utf8_lcase#x as string), Some(collate(AAa, utf8_binary))) AS TRIM(TRAILING collate(AAa, utf8_binary) FROM utf8_lcase)#x] ++- SubqueryAlias spark_catalog.default.t5 + +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet + + +-- !query +drop table t5 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t5 -- !query @@ -576,3 +2319,31 @@ drop table t6 -- !query analysis DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t6 + + +-- !query +drop table t7 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t7 + + +-- !query +drop table t8 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t8 + + +-- !query +drop table t9 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t9 + + +-- !query +drop table t10 +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t10 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 51d8d1be4154c..183577b83971b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -102,27 +102,307 @@ select str_to_map(text collate utf8_binary, pairDelim collate utf8_binary, keyVa drop table t4; --- create table for split_part -create table t5(str string collate utf8_binary, delimiter string collate utf8_lcase, partNum int) using parquet; - -insert into t5 values('11AB12AB13', 'AB', 2); - -select split_part(str, delimiter, partNum) from t5; -select split_part(str collate utf8_binary, delimiter collate utf8_lcase, partNum) from t5; -select split_part(str collate utf8_binary, delimiter collate utf8_binary, partNum) from t5; +create table t5(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t5 values ('Spark', 'Spark', 'SQL'); +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA'); +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA'); +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA'); +insert into t5 values ('bbAbaAbA', 'bbAbAAbA', 'a'); +insert into t5 values ('İo', 'İo', 'İo'); +insert into t5 values ('İo', 'İo', 'i̇o'); +insert into t5 values ('efd2', 'efd2', 'efd2'); +insert into t5 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.'); +insert into t5 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.'); +insert into t5 values ('kitten', 'kitten', 'sitTing'); +insert into t5 values ('abc', 'abc', 'abc'); +insert into t5 values ('abcdcba', 'abcdcba', 'aBcDCbA'); + +create table t6(ascii long) using parquet; +insert into t6 values (97); +insert into t6 values (66); + +create table t7(ascii double) using parquet; +insert into t7 values (97.52143); +insert into t7 values (66.421); + +create table t8(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t8 values ('%s%s', 'abCdE', 'abCdE'); + +create table t9(num long) using parquet; +insert into t9 values (97); +insert into t9 values (66); + +create table t10(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t10 values ('aaAaAAaA', 'aaAaaAaA'); +insert into t10 values ('efd2', 'efd2'); + +-- ConcatWs +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5; +select concat_ws(' ', utf8_binary, utf8_lcase) from t5; +select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t5; +select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t5; +select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5; +select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5; + +-- Elt +select elt(2, s, utf8_binary) from t5; +select elt(2, utf8_binary, utf8_lcase, s) from t5; +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t5; +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5; +select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5; +select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5; +select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5; + +-- SplitPart +select split_part(utf8_binary, utf8_lcase, 3) from t5; +select split_part(s, utf8_binary, 1) from t5; +select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t5; +select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5; +select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5; +select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5; +select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5; + +-- Contains +select contains(utf8_binary, utf8_lcase) from t5; +select contains(s, utf8_binary) from t5; +select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select contains(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5; +select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t5; + +-- SubstringIndex +select substring_index(utf8_binary, utf8_lcase, 2) from t5; +select substring_index(s, utf8_binary,1) from t5; +select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5; +select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5; +select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5; +select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5; +select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5; + +-- StringInStr +select instr(utf8_binary, utf8_lcase) from t5; +select instr(s, utf8_binary) from t5; +select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select instr(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5; +select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t5; + +-- FindInSet +select find_in_set(utf8_binary, utf8_lcase) from t5; +select find_in_set(s, utf8_binary) from t5; +select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5; +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5; + +-- StartsWith +select startswith(utf8_binary, utf8_lcase) from t5; +select startswith(s, utf8_binary) from t5; +select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5; +select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5; + +-- StringTranslate +select translate(utf8_lcase, utf8_lcase, '12345') from t5; +select translate(utf8_binary, utf8_lcase, '12345') from t5; +select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t5; +select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t5; +select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5; +select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5; + +-- Replace +select replace(utf8_binary, utf8_lcase, 'abc') from t5; +select replace(s, utf8_binary, 'abc') from t5; +select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t5; +select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t5; +select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5; +select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5; +select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5; + +-- EndsWith +select endswith(utf8_binary, utf8_lcase) from t5; +select endswith(s, utf8_binary) from t5; +select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5; +select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5; + +-- StringRepeat +select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t5; +select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t5; + +-- Ascii & UnBase64 string expressions +select ascii(utf8_binary), ascii(utf8_lcase) from t5; +select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t5; +select unbase64(utf8_binary), unbase64(utf8_lcase) from t10; +select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t10; + +-- Chr +select chr(ascii) from t6; + +-- Base64, Decode +select base64(utf8_binary), base64(utf8_lcase) from t5; +select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t5; +select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t5; +select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t5; + +-- FormatNumber +select format_number(ascii, '###.###') from t7; +select format_number(ascii, '###.###' collate utf8_lcase) from t7; + +-- Encode, ToBinary +select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t5; +select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t5; +select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t5; +select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t5; + +-- Sentences +select sentences(utf8_binary), sentences(utf8_lcase) from t5; +select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t5; + +-- Upper +select upper(utf8_binary), upper(utf8_lcase) from t5; +select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t5; + +-- Lower +select lower(utf8_binary), lower(utf8_lcase) from t5; +select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t5; + +-- InitCap +select initcap(utf8_binary), initcap(utf8_lcase) from t5; +select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t5; + +-- Overlay +select overlay(utf8_binary, utf8_lcase, 2) from t5; +select overlay(s, utf8_binary,1) from t5; +select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5; +select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5; +select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5; +select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5; +select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5; + +-- FormatString +select format_string(format, utf8_binary, utf8_lcase) from t8; +select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t8; +select format_string(format, utf8_binary, utf8_lcase) from t8; + +-- SoundEx +select soundex(utf8_binary), soundex(utf8_lcase) from t5; +select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t5; + +-- Length, BitLength & OctetLength +select length(utf8_binary), length(utf8_lcase) from t5; +select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t5; +select bit_length(utf8_binary), bit_length(utf8_lcase) from t5; +select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5; +select octet_length(utf8_binary), octet_length(utf8_lcase) from t5; +select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5; + +-- Luhncheck +select luhn_check(num) from t9; + +-- Levenshtein +select levenshtein(utf8_binary, utf8_lcase) from t5; +select levenshtein(s, utf8_binary) from t5; +select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5; +select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5; + +-- IsValidUTF8 +select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5; +select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5; + +-- MakeValidUTF8 +select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5; +select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5; + +-- ValidateUTF8 +select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5; +select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5; + +-- TryValidateUTF8 +select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5; +select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5; + +-- Left/Right/Substr +select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5; +select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t5; +select right(utf8_binary, 2), right(utf8_lcase, 2) from t5; +select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t5; +select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t5; +select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t5; + +-- StringRPad +select rpad(utf8_binary, 8, utf8_lcase) from t5; +select rpad(s, 8, utf8_binary) from t5; +select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5; +select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5; +select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5; +select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5; +select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5; + +-- StringLPad +select lpad(utf8_binary, 8, utf8_lcase) from t5; +select lpad(s, 8, utf8_binary) from t5; +select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5; +select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5; +select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5; +select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5; +select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5; + +-- Locate +select locate(utf8_binary, utf8_lcase) from t5; +select locate(s, utf8_binary) from t5; +select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select locate(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t5; +select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5; +select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5; + +-- StringTrim +select TRIM(utf8_binary, utf8_lcase) from t5; +select TRIM(s, utf8_binary) from t5; +select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5; +select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t5; +-- StringTrimBoth +select BTRIM(utf8_binary, utf8_lcase) from t5; +select BTRIM(s, utf8_binary) from t5; +select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t5; +select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t5; +-- StringTrimLeft +select LTRIM(utf8_binary, utf8_lcase) from t5; +select LTRIM(s, utf8_binary) from t5; +select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5; +select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t5; +-- StringTrimRight +select RTRIM(utf8_binary, utf8_lcase) from t5; +select RTRIM(s, utf8_binary) from t5; +select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5; +select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t5; drop table t5; - --- create table for levenshtein -create table t6 (utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase, threshold int) using parquet; - -insert into t6 values('kitten', 'sitting', 2); - -select levenshtein(utf8_binary, utf8_lcase) from t6; -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t6; -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t6; -select levenshtein(utf8_binary, utf8_lcase, threshold) from t6; -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase, threshold) from t6; -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary, threshold) from t6; - drop table t6; +drop table t7; +drop table t8; +drop table t9; +drop table t10; 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 d8f8d0676baed..ea5564aafe96f 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 @@ -489,7 +489,7 @@ struct<> -- !query -create table t5(str string collate utf8_binary, delimiter string collate utf8_lcase, partNum int) using parquet +create table t5(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet -- !query schema struct<> -- !query output @@ -497,7 +497,7 @@ struct<> -- !query -insert into t5 values('11AB12AB13', 'AB', 2) +insert into t5 values ('Spark', 'Spark', 'SQL') -- !query schema struct<> -- !query output @@ -505,7 +505,3113 @@ struct<> -- !query -select split_part(str, delimiter, partNum) from t5 +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('bbAbaAbA', 'bbAbAAbA', 'a') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('İo', 'İo', 'İo') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('İo', 'İo', 'i̇o') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('efd2', 'efd2', 'efd2') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('kitten', 'kitten', 'sitTing') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('abc', 'abc', 'abc') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('abcdcba', 'abcdcba', 'aBcDCbA') +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t6(ascii long) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t6 values (97) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t6 values (66) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t7(ascii double) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t7 values (97.52143) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t7 values (66.421) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t8(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t8 values ('%s%s', 'abCdE', 'abCdE') +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t9(num long) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t9 values (97) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t9 values (66) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t10(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t10 values ('aaAaAAaA', 'aaAaaAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t10 values ('efd2', 'efd2') +-- !query schema +struct<> +-- !query output + + + +-- !query +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo + + +-- !query +select concat_ws(' ', utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. SQL +Something else. Nothing here. SQL +Spark SQL +aaAaAAaA SQL +aaAaAAaA SQL +aaAaAAaA SQL +abc SQL +abcdcba SQL +bbAbAAbA SQL +efd2 SQL +kitten SQL +İo SQL +İo SQL + + +-- !query +select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.,word Hello, world! Nice day.,word +SQL,word Spark,word +Something else. Nothing here.,word Something else. Nothing here.,word +a,word bbAbAAbA,word +aBcDCbA,word abcdcba,word +aaAaAAaA,word aaAaAAaA,word +aaAaaAaA,word aaAaAAaA,word +aaAaaAaAaaAaaAaAaaAaaAaA,word aaAaAAaA,word +abc,word abc,word +efd2,word efd2,word +i̇o,word İo,word +sitTing,word kitten,word +İo,word İo,word + + +-- !query +select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.,word Hello, world! Nice day.,word +SQL,word Spark,word +Something else. Nothing here.,word Something else. Nothing here.,word +a,word bbAbAAbA,word +aBcDCbA,word abcdcba,word +aaAaAAaA,word aaAaAAaA,word +aaAaaAaA,word aaAaAAaA,word +aaAaaAaAaaAaaAaAaaAaaAaA,word aaAaAAaA,word +abc,word abc,word +efd2,word efd2,word +i̇o,word İo,word +sitTing,word kitten,word +İo,word İo,word + + +-- !query +select elt(2, s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo + + +-- !query +select elt(2, utf8_binary, utf8_lcase, s) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo + + +-- !query +select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select split_part(utf8_binary, utf8_lcase, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select split_part(s, utf8_binary, 1) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + +bbAbaAbA + + +-- !query +select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + +b + + +-- !query +select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +A +A +A + + +-- !query +select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + A + A + A + + +-- !query +select contains(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select contains(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +true +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select contains(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +false +false +true +true +true +true +true +true + + +-- !query +select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +false +false +false +true +true +true +true +true +true +true +true +true +true + + +-- !query +select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false true +true false +true true +true true +true true +true true +true true +true true + + +-- !query +select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true false +true true + + +-- !query +select substring_index(utf8_binary, utf8_lcase, 2) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select substring_index(s, utf8_binary,1) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + +bbAbaAbA + + +-- !query +select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo + + +-- !query +select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAb +efd2 +kitten +İo +İo + + +-- !query +select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +a a +a a +a a +abc abc +abcdcb aBcDCb +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +a aaAaAAaA +a aaAaaAaA +a aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select instr(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select instr(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select instr(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 + + +-- !query +select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +3 + + +-- !query +select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 +1 1 +1 1 +1 1 +1 1 +1 1 +21 21 +3 0 + + +-- !query +select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 0 +1 0 +1 5 + + +-- !query +select find_in_set(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select find_in_set(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 + + +-- !query +select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 2 +0 2 + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 0 +1 0 +1 1 +2 0 +2 2 + + +-- !query +select startswith(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select startswith(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +true +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +false +false +true +true +true +true +true +true + + +-- !query +select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +true +true +true +true +true +true +true +true +true + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false true +false true +false true + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true true +true true + + +-- !query +select translate(utf8_lcase, utf8_lcase, '12345') from t5 +-- !query schema +struct +-- !query output +1 +11111111 +11111111 +111111111111111111111111 +12 +123 +123 +123 +12332 +12335532 +1234 +1234321 +123454142544 + + +-- !query +select translate(utf8_binary, utf8_lcase, '12345') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" + } +} + + +-- !query +select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +1omething e31e. Nothing here. +1park +He33o, wor3d! Nice day. +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo + + +-- !query +select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5 +-- !query schema +struct +-- !query output +1 bb3b33b3 +11111111 11313313 +11111111 11313313 +111111111111111111111111 11313313 +1BcDCb1 1bcdcb1 +1bc 1bc +Hello, world! Nice d1y. Hello, world! Nice d1y. +SQL Sp1rk +Something else. Nothing here. Something else. Nothing here. +efd2 efd2 +i̇o İo +sitTing kitten +İo İo + + +-- !query +select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5 +-- !query schema +struct +-- !query output +1 22121121 +11A11A1A 11111111 +11A11A1A11A11A1A11A11A1A 11111111 +11A1AA1A 11111111 +123DCbA 123d321 +1b3 123 +Hello, world! Ni3e d1y. Hello, world! Ni3e d1y. +SQL Sp1rk +Something else. Nothing here. Something else. Nothing here. +efd2 efd2 +i̇o İo +sitTing kitten +İo İo + + +-- !query +select replace(utf8_binary, utf8_lcase, 'abc') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select replace(s, utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +bbAbaAbA + + +-- !query +select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +Spark +aaAaAAaA +aaAaAAaA +abc +abc +abc +abc +abc +abc +abcdcba +bbAbAAbA +kitten +İo + + +-- !query +select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5 +-- !query schema +struct +-- !query output +Spark +aaAaAAaA +abc +abc +abc +abc +abc +abc +abc +abc +abc +bbabcbabcabcbabc +kitten + + +-- !query +select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA abc +aaAaAAaA abc +aaAaAAaA abcabcabc +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +abc aaAaAAaA +abc abc +abc abc +abc abcabcabc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select endswith(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select endswith(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +true +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +false +false +true +true +true +true +true +true + + +-- !query +select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +true +true +true +true +true +true +true +true + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false true +false true +false true + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true true +true true + + +-- !query +select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.Hello, world! Nice day.Hello, world! Nice day. Hello, world! Nice day.Hello, world! Nice day. +Something else. Nothing here.Something else. Nothing here.Something else. Nothing here. Something else. Nothing here.Something else. Nothing here. +SparkSparkSpark SQLSQL +aaAaAAaAaaAaAAaAaaAaAAaA aaAaAAaAaaAaAAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaA +abcabcabc abcabc +abcdcbaabcdcbaabcdcba aBcDCbAaBcDCbA +bbAbAAbAbbAbAAbAbbAbAAbA aa +efd2efd2efd2 efd2efd2 +kittenkittenkitten sitTingsitTing +İoİoİo i̇oi̇o +İoİoİo İoİo + + +-- !query +select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.Hello, world! Nice day.Hello, world! Nice day. Hello, world! Nice day.Hello, world! Nice day. +Something else. Nothing here.Something else. Nothing here.Something else. Nothing here. Something else. Nothing here.Something else. Nothing here. +SparkSparkSpark SQLSQL +aaAaAAaAaaAaAAaAaaAaAAaA aaAaAAaAaaAaAAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaA +abcabcabc abcabc +abcdcbaabcdcbaabcdcba aBcDCbAaBcDCbA +bbAbAAbAbbAbAAbAbbAbAAbA aa +efd2efd2efd2 efd2efd2 +kittenkittenkitten sitTingsitTing +İoİoİo i̇oi̇o +İoİoİo İoİo + + +-- !query +select ascii(utf8_binary), ascii(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +101 101 +107 115 +304 105 +304 304 +72 72 +83 83 +83 83 +97 97 +97 97 +97 97 +97 97 +97 97 +98 97 + + +-- !query +select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +101 101 +107 115 +304 105 +304 304 +72 72 +83 83 +83 83 +97 97 +97 97 +97 97 +97 97 +97 97 +98 97 + + +-- !query +select unbase64(utf8_binary), unbase64(utf8_lcase) from t10 +-- !query schema +struct +-- !query output +i�� i�h� +y�v y�v + + +-- !query +select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t10 +-- !query schema +struct +-- !query output +i�� i�h� +y�v y�v + + +-- !query +select chr(ascii) from t6 +-- !query schema +struct +-- !query output +B +a + + +-- !query +select base64(utf8_binary), base64(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= +U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= +U3Bhcms= U1FM +YWFBYUFBYUE= YWFBYUFBYUE= +YWFBYUFBYUE= YWFBYWFBYUE= +YWFBYUFBYUE= YWFBYWFBYUFhYUFhYUFhQWFhQWFhQWFB +YWJj YWJj +YWJjZGNiYQ== YUJjRENiQQ== +YmJBYkFBYkE= YQ== +ZWZkMg== ZWZkMg== +a2l0dGVu c2l0VGluZw== +xLBv acyHbw== +xLBv xLBv + + +-- !query +select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= +U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= +U3Bhcms= U1FM +YWFBYUFBYUE= YWFBYUFBYUE= +YWFBYUFBYUE= YWFBYWFBYUE= +YWFBYUFBYUE= YWFBYWFBYUFhYUFhYUFhQWFhQWFhQWFB +YWJj YWJj +YWJjZGNiYQ== YUJjRENiQQ== +YmJBYkFBYkE= YQ== +ZWZkMg== ZWZkMg== +a2l0dGVu c2l0VGluZw== +xLBv acyHbw== +xLBv xLBv + + +-- !query +select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select format_number(ascii, '###.###') from t7 +-- !query schema +struct +-- !query output +66.421 +97.521 + + +-- !query +select format_number(ascii, '###.###' collate utf8_lcase) from t7 +-- !query schema +struct +-- !query output +66.421 +97.521 + + +-- !query +select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select sentences(utf8_binary), sentences(utf8_lcase) from t5 +-- !query schema +struct>,sentences(utf8_lcase, , ):array>> +-- !query output +[["Hello","world"],["Nice","day"]] [["Hello","world"],["Nice","day"]] +[["Something","else"],["Nothing","here"]] [["Something","else"],["Nothing","here"]] +[["Spark"]] [["SQL"]] +[["aaAaAAaA"]] [["aaAaAAaA"]] +[["aaAaAAaA"]] [["aaAaaAaA"]] +[["aaAaAAaA"]] [["aaAaaAaAaaAaaAaAaaAaaAaA"]] +[["abc"]] [["abc"]] +[["abcdcba"]] [["aBcDCbA"]] +[["bbAbAAbA"]] [["a"]] +[["efd2"]] [["efd2"]] +[["kitten"]] [["sitTing"]] +[["İo"]] [["i̇o"]] +[["İo"]] [["İo"]] + + +-- !query +select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct>,sentences(collate(utf8_lcase, utf8_binary), , ):array>> +-- !query output +[["Hello","world"],["Nice","day"]] [["Hello","world"],["Nice","day"]] +[["Something","else"],["Nothing","here"]] [["Something","else"],["Nothing","here"]] +[["Spark"]] [["SQL"]] +[["aaAaAAaA"]] [["aaAaAAaA"]] +[["aaAaAAaA"]] [["aaAaaAaA"]] +[["aaAaAAaA"]] [["aaAaaAaAaaAaaAaAaaAaaAaA"]] +[["abc"]] [["abc"]] +[["abcdcba"]] [["aBcDCbA"]] +[["bbAbAAbA"]] [["a"]] +[["efd2"]] [["efd2"]] +[["kitten"]] [["sitTing"]] +[["İo"]] [["i̇o"]] +[["İo"]] [["İo"]] + + +-- !query +select upper(utf8_binary), upper(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAAAAAAAAAAAAAAAAAA +ABC ABC +ABCDCBA ABCDCBA +BBABAABA A +EFD2 EFD2 +HELLO, WORLD! NICE DAY. HELLO, WORLD! NICE DAY. +KITTEN SITTING +SOMETHING ELSE. NOTHING HERE. SOMETHING ELSE. NOTHING HERE. +SPARK SQL +İO İO +İO İO + + +-- !query +select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAAAAAAAAAAAAAAAAAA +ABC ABC +ABCDCBA ABCDCBA +BBABAABA A +EFD2 EFD2 +HELLO, WORLD! NICE DAY. HELLO, WORLD! NICE DAY. +KITTEN SITTING +SOMETHING ELSE. NOTHING HERE. SOMETHING ELSE. NOTHING HERE. +SPARK SQL +İO İO +İO İO + + +-- !query +select lower(utf8_binary), lower(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaaaaaaaaaaaaaaaaaa +abc abc +abcdcba abcdcba +bbabaaba a +efd2 efd2 +hello, world! nice day. hello, world! nice day. +i̇o i̇o +i̇o i̇o +kitten sitting +something else. nothing here. something else. nothing here. +spark sql + + +-- !query +select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaaaaaaaaaaaaaaaaaa +abc abc +abcdcba abcdcba +bbabaaba a +efd2 efd2 +hello, world! nice day. hello, world! nice day. +i̇o i̇o +i̇o i̇o +kitten sitting +something else. nothing here. something else. nothing here. +spark sql + + +-- !query +select initcap(utf8_binary), initcap(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaaaaaaaaaaaaaaaaaa +Abc Abc +Abcdcba Abcdcba +Bbabaaba A +Efd2 Efd2 +Hello, World! Nice Day. Hello, World! Nice Day. +Kitten Sitting +Something Else. Nothing Here. Something Else. Nothing Here. +Spark Sql +İo İo +İo İo + + +-- !query +select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaaaaaaaaaaaaaaaaaa +Abc Abc +Abcdcba Abcdcba +Bbabaaba A +Efd2 Efd2 +Hello, World! Nice Day. Hello, World! Nice Day. +Kitten Sitting +Something Else. Nothing Here. Something Else. Nothing Here. +Spark Sql +İo İo +İo İo + + +-- !query +select overlay(utf8_binary, utf8_lcase, 2) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select overlay(s, utf8_binary,1) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo + + +-- !query +select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +HHello, world! Nice day. +SSQLk +SSomething else. Nothing here. +aaBcDCbA +aaaAaAAaA +aaaAaaAaA +aaaAaaAaAaaAaaAaAaaAaaAaA +aabc +baAbAAbA +eefd2 +ksitTing +İi̇o +İİo + + +-- !query +select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +HHello, world! Nice day. +SSQLk +SSomething else. Nothing here. +aaBcDCbA +aaaAaAAaA +aaaAaaAaA +aaaAaaAaAaaAaaAaAaaAaaAaA +aabc +baAbAAbA +eefd2 +ksitTing +İi̇o +İİo + + +-- !query +select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5 +-- !query schema +struct +-- !query output +Hallo, world! Nice day. Hallo, world! Nice day. +Saark SaL +Samething else. Nothing here. Samething else. Nothing here. +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +aac aac +aacdcba aacDCbA +baAbAAbA aa +ead2 ead2 +katten satTing +İa iao +İa İa + + +-- !query +select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +HAaAA, world! Nice day. HAAao, world! Nice day. +SAaAA SAAa +SAaAAhing else. Nothing here. SAAathing else. Nothing here. +aAaAA aAAa +aAaAAAaA aAAaAAaA +aAaAAAaA aAAaaAaA +aAaAAAaA aAAaaAaAaaAaaAaAaaAaaAaA +aAaAAba aAAaCbA +bAaAAAbA aAAa +eAaAA eAAa +kAaAAn sAAaing +İAaAA iAAa +İAaAA İAAa + + +-- !query +select format_string(format, utf8_binary, utf8_lcase) from t8 +-- !query schema +struct +-- !query output +abCdEabCdE + + +-- !query +select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t8 +-- !query schema +struct +-- !query output +abCdEabCdE abCdEabCdE + + +-- !query +select format_string(format, utf8_binary, utf8_lcase) from t8 +-- !query schema +struct +-- !query output +abCdEabCdE + + +-- !query +select soundex(utf8_binary), soundex(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +A000 A000 +A000 A000 +A000 A000 +A120 A120 +A123 A123 +B110 A000 +E130 E130 +H464 H464 +K350 S352 +S162 S400 +S535 S535 +İo I000 +İo İo + + +-- !query +select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +A000 A000 +A000 A000 +A000 A000 +A120 A120 +A123 A123 +B110 A000 +E130 E130 +H464 H464 +K350 S352 +S162 S400 +S535 S535 +İo I000 +İo İo + + +-- !query +select length(utf8_binary), length(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +2 2 +2 3 +23 23 +29 29 +3 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +2 2 +2 3 +23 23 +29 29 +3 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select bit_length(utf8_binary), bit_length(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +184 184 +232 232 +24 24 +24 24 +24 32 +32 32 +40 24 +48 56 +56 56 +64 192 +64 64 +64 64 +64 8 + + +-- !query +select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +184 184 +232 232 +24 24 +24 24 +24 32 +32 32 +40 24 +48 56 +56 56 +64 192 +64 64 +64 64 +64 8 + + +-- !query +select octet_length(utf8_binary), octet_length(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +23 23 +29 29 +3 3 +3 3 +3 4 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +23 23 +29 29 +3 3 +3 3 +3 4 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select luhn_check(num) from t9 +-- !query schema +struct +-- !query output +false +false + + +-- !query +select levenshtein(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select levenshtein(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 + + +-- !query +select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +1 +16 +2 +4 +4 +4 +8 + + +-- !query +select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +1 +16 +2 +4 +4 +4 +8 + + +-- !query +select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +2 2 +2 2 +2 3 +22 22 +29 29 +4 3 +4 4 +6 6 +6 7 +7 23 +7 7 +7 7 +8 0 + + +-- !query +select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5 +-- !query schema +struct +-- !query output +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 2 +-1 3 +-1 3 +-1 3 +-1 4 +3 3 + + +-- !query +select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true + + +-- !query +select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true + + +-- !query +select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA +bc Bc +bc bc +el el +fd fd +it it +o o +o ̇o +om om +pa QL + + +-- !query +select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA +bc Bc +bc bc +el el +fd fd +it it +o o +o ̇o +om om +pa QL + + +-- !query +select right(utf8_binary, 2), right(utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA a +ba bA +bc bc +d2 d2 +e. e. +en ng +rk QL +y. y. +İo İo +İo ̇o + + +-- !query +select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA a +ba bA +bc bc +d2 d2 +e. e. +en ng +rk QL +y. y. +İo İo +İo ̇o + + +-- !query +select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +He He +So So +Sp SQ +aa aa +aa aa +aa aa +ab aB +ab ab +bb a +ef ef +ki si +İo i̇ +İo İo + + +-- !query +select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +He He +So So +Sp SQ +aa aa +aa aa +aa aa +ab aB +ab ab +bb a +ef ef +ki si +İo i̇ +İo İo + + +-- !query +select rpad(utf8_binary, 8, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select rpad(s, 8, utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSpa +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbaAbA +efd2efd2 +kittenki +İoİoİoİo +İoİoİoİo + + +-- !query +select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbAAbA +efd2efd2 +kittensi +İoi̇oi̇o +İoİoİoİo + + +-- !query +select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbAAbA +efd2efd2 +kittensi +İoi̇oi̇o +İoİoİoİo + + +-- !query +select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5 +-- !query schema +struct +-- !query output +Hello, w Hello, w +Somethin Somethin +Sparkaaa SQLaaaaa +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +abcaaaaa abcaaaaa +abcdcbaa aBcDCbAa +bbAbAAbA aaaaaaaa +efd2aaaa efd2aaaa +kittenaa sitTinga +İoaaaaaa i̇oaaaaa +İoaaaaaa İoaaaaaa + + +-- !query +select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w Hello, w +Somethin Somethin +SparkAaA SQLAAaAA +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +abcAaAAA abcAAaAA +abcdcbaA aBcDCbAA +bbAbAAbA aAAaAAaA +efd2AaAA efd2AAaA +kittenAa sitTingA +İoAaAAAa i̇oAAaAA +İoAaAAAa İoAAaAAa + + +-- !query +select lpad(utf8_binary, 8, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select lpad(s, 8, utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SpaSpark +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbaAbA +efd2efd2 +kikitten +İoİoİoİo +İoİoİoİo + + +-- !query +select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İoİoİoİo + + +-- !query +select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İoİoİoİo + + +-- !query +select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5 +-- !query schema +struct +-- !query output +Hello, w Hello, w +Somethin Somethin +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +aaaSpark aaaaaSQL +aaaaaabc aaaaaabc +aaaaaaİo aaaaaaİo +aaaaaaİo aaaaai̇o +aaaaefd2 aaaaefd2 +aabcdcba aaBcDCbA +aakitten asitTing +bbAbAAbA aaaaaaaa + + +-- !query +select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +AaAAAabc AAaAAabc +AaAAAaİo AAaAAaİo +AaAAAaİo AAaAAi̇o +AaAAefd2 AAaAefd2 +AaASpark AAaAASQL +Aabcdcba AaBcDCbA +Aakitten AsitTing +Hello, w Hello, w +Somethin Somethin +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +bbAbAAbA AAaAAaAa + + +-- !query +select locate(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select locate(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select locate(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 + + +-- !query +select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +3 + + +-- !query +select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 + + +-- !query +select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 + + +-- !query +select TRIM(utf8_binary, utf8_lcase) from t5 -- !query schema struct<> -- !query output @@ -517,7 +3623,15 @@ org.apache.spark.sql.AnalysisException -- !query -select split_part(str collate utf8_binary, delimiter collate utf8_lcase, partNum) from t5 +select TRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 -- !query schema struct<> -- !query output @@ -526,45 +3640,220 @@ org.apache.spark.sql.AnalysisException "errorClass" : "COLLATION_MISMATCH.EXPLICIT", "sqlState" : "42P21", "messageParameters" : { - "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" } } -- !query -select split_part(str collate utf8_binary, delimiter collate utf8_binary, partNum) from t5 +select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 -- !query schema -struct +struct -- !query output -12 + + + + + + + + +BcDCbA +QL +a +i̇ +sitTing -- !query -drop table t5 +select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 -- !query schema -struct<> +struct +-- !query output + + + + + + + + + + + +QL +sitTing + + +-- !query +select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct -- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAa +aaAaAAa +aaAaAAa +ab +abcdcba D +bbAbAAb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + +-- !query +select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + bc +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +d BcDCb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo -- !query -create table t6 (utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase, threshold int) using parquet +select BTRIM(utf8_binary, utf8_lcase) from t5 -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.IMPLICIT", + "sqlState" : "42P21" +} + + +-- !query +select BTRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + +a + + -- !query -insert into t6 values('kitten', 'sitting', 2) +select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 -- !query schema struct<> -- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + } +} + + +-- !query +select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + +bbAbAAbA +d +kitte +park +İ + + +-- !query +select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + +bbAbAAb +kitte +park +İ + + +-- !query +select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +AB +AB +AB B +ABc ABc +ABc ABc +ABc ABc +ABc ABc +ABc ABc +ABc ABc +Bc Bc +Bc Bc +Bc Bc +Bc Bc + +-- !query +select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + AA +ABc AAa +ABc AAa +ABc AAa +ABc AAa +ABc AAa +B AA +Bc +Bc +Bc +Bc AAa +c AA -- !query -select levenshtein(utf8_binary, utf8_lcase) from t6 +select LTRIM(utf8_binary, utf8_lcase) from t5 -- !query schema struct<> -- !query output @@ -576,7 +3865,15 @@ org.apache.spark.sql.AnalysisException -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t6 +select LTRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 -- !query schema struct<> -- !query output @@ -585,21 +3882,93 @@ org.apache.spark.sql.AnalysisException "errorClass" : "COLLATION_MISMATCH.EXPLICIT", "sqlState" : "42P21", "messageParameters" : { - "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" } } -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t6 +select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 -- !query schema -struct +struct -- !query output -3 + + + + + + + + +BcDCbA +QL +a +i̇o +sitTing + + +-- !query +select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + +QL +sitTing + + +-- !query +select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba DCbA +bbAbAAbA +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + bc +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +dcba BcDCbA +efd2 efd2 +kitten sitTing +İo i̇o +İo İo -- !query -select levenshtein(utf8_binary, utf8_lcase, threshold) from t6 +select RTRIM(utf8_binary, utf8_lcase) from t5 -- !query schema struct<> -- !query output @@ -611,7 +3980,15 @@ org.apache.spark.sql.AnalysisException -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase, threshold) from t6 +select RTRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 -- !query schema struct<> -- !query output @@ -620,17 +3997,97 @@ org.apache.spark.sql.AnalysisException "errorClass" : "COLLATION_MISMATCH.EXPLICIT", "sqlState" : "42P21", "messageParameters" : { - "explicitTypes" : "`string`, `string collate UTF8_LCASE`" + "explicitTypes" : "`string collate UTF8_LCASE`, `string`" } } -- !query -select levenshtein(utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary, threshold) from t6 +select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + +SQL +a +aBcDCbA +i̇ +sitTing + + +-- !query +select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + +SQL +sitTing + + +-- !query +select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAa +aaAaAAa +aaAaAAa +ab +abcdcba aBcD +bbAbAAb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + abc +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +abcd aBcDCb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo + + +-- !query +drop table t5 -- !query schema -struct +struct<> -- !query output --1 + -- !query @@ -639,3 +4096,35 @@ drop table t6 struct<> -- !query output + + +-- !query +drop table t7 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t8 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t9 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t10 +-- !query schema +struct<> +-- !query output + 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 47a6143bad1d7..370c118de9a93 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 @@ -24,6 +24,7 @@ import org.apache.spark.sql._ 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.SaveIntoDataSourceCommand import org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions._ @@ -926,6 +927,25 @@ class QueryCompilationErrorsSuite }) } } + + test("Catch and log errors when failing to write to external data source") { + val password = "MyPassWord" + val token = "MyToken" + val value = "value" + val options = Map("password" -> password, "token" -> token, "key" -> value) + val query = spark.range(10).logicalPlan + val cmd = SaveIntoDataSourceCommand(query, null, options, SaveMode.Overwrite) + + checkError( + exception = intercept[AnalysisException] { + cmd.run(spark) + }, + condition = "DATA_SOURCE_EXTERNAL_ERROR", + sqlState = "KD00F", + parameters = Map.empty + ) + } + } class MyCastToString extends SparkUserDefinedFunction( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala index 91b1bfd7bf213..62a32da22d957 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExecuteImmediateEndToEndSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{QueryTest} -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSparkSession class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession { @@ -37,30 +36,4 @@ class ExecuteImmediateEndToEndSuite extends QueryTest with SharedSparkSession { spark.sql("DROP TEMPORARY VARIABLE IF EXISTS parm;") } } - - test("EXEC IMMEDIATE STACK OVERFLOW") { - try { - spark.sql("DECLARE parm = 1;") - val query = (1 to 20000).map(x => "SELECT 1 as a").mkString(" UNION ALL ") - Seq( - s"EXECUTE IMMEDIATE '$query'", - s"EXECUTE IMMEDIATE '$query' INTO parm").foreach { q => - val e = intercept[ParseException] { - spark.sql(q) - } - - checkError( - exception = e, - condition = "FAILED_TO_PARSE_TOO_COMPLEX", - parameters = Map(), - context = ExpectedContext( - query, - start = 0, - stop = query.length - 1) - ) - } - } finally { - spark.sql("DROP TEMPORARY VARIABLE IF EXISTS parm;") - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala index 4b72ca8ecaa97..83d8191d01ec1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNodeSuite.scala @@ -576,4 +576,97 @@ class SqlScriptingExecutionNodeSuite extends SparkFunSuite with SharedSparkSessi "body1", "lbl", "con1", "body1", "lbl", "con1")) } + + test("searched case - enter first WHEN clause") { + val iter = new CompoundBodyExec(Seq( + new CaseStatementExec( + conditions = Seq( + TestIfElseCondition(condVal = true, description = "con1"), + TestIfElseCondition(condVal = false, description = "con2") + ), + conditionalBodies = Seq( + new CompoundBodyExec(Seq(TestLeafStatement("body1"))), + new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + ), + elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + session = spark + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("con1", "body1")) + } + + test("searched case - enter body of the ELSE clause") { + val iter = new CompoundBodyExec(Seq( + new CaseStatementExec( + conditions = Seq( + TestIfElseCondition(condVal = false, description = "con1") + ), + conditionalBodies = Seq( + new CompoundBodyExec(Seq(TestLeafStatement("body1"))) + ), + elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body2")))), + session = spark + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("con1", "body2")) + } + + test("searched case - enter second WHEN clause") { + val iter = new CompoundBodyExec(Seq( + new CaseStatementExec( + conditions = Seq( + TestIfElseCondition(condVal = false, description = "con1"), + TestIfElseCondition(condVal = true, description = "con2") + ), + conditionalBodies = Seq( + new CompoundBodyExec(Seq(TestLeafStatement("body1"))), + new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + ), + elseBody = Some(new CompoundBodyExec(Seq(TestLeafStatement("body3")))), + session = spark + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("con1", "con2", "body2")) + } + + test("searched case - without else (successful check)") { + val iter = new CompoundBodyExec(Seq( + new CaseStatementExec( + conditions = Seq( + TestIfElseCondition(condVal = false, description = "con1"), + TestIfElseCondition(condVal = true, description = "con2") + ), + conditionalBodies = Seq( + new CompoundBodyExec(Seq(TestLeafStatement("body1"))), + new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + ), + elseBody = None, + session = spark + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("con1", "con2", "body2")) + } + + test("searched case - without else (unsuccessful checks)") { + val iter = new CompoundBodyExec(Seq( + new CaseStatementExec( + conditions = Seq( + TestIfElseCondition(condVal = false, description = "con1"), + TestIfElseCondition(condVal = false, description = "con2") + ), + conditionalBodies = Seq( + new CompoundBodyExec(Seq(TestLeafStatement("body1"))), + new CompoundBodyExec(Seq(TestLeafStatement("body2"))) + ), + elseBody = None, + session = spark + ) + )).getTreeIterator + val statements = iter.map(extractStatementValue).toSeq + assert(statements === Seq("con1", "con2")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 8d9cd1d8c780e..3fad99eba509a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.scripting -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, SparkNumberFormatException} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row} import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.exceptions.SqlScriptingException @@ -368,6 +368,383 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } } + test("searched case") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("searched case nested") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1=1 THEN + | CASE + | WHEN 2=1 THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END CASE; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("searched case second case") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = (SELECT 2) THEN + | SELECT 1; + | WHEN 2 = 2 THEN + | SELECT 42; + | WHEN (SELECT * FROM t) THEN + | SELECT * FROM b; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("searched case going in else") { + val commands = + """ + |BEGIN + | CASE + | WHEN 2 = 1 THEN + | SELECT 1; + | WHEN 3 IN (1,2) THEN + | SELECT 2; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + + test("searched case with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |INSERT INTO t VALUES (1, 'a', 1.0); + |CASE + | WHEN (SELECT COUNT(*) > 2 FROM t) THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + + val expected = Seq(Seq.empty[Row], Seq.empty[Row], Seq.empty[Row], Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("searched case else with count") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | INSERT INTO t VALUES (1, 'a', 1.0); + | CASE + | WHEN (SELECT COUNT(*) > 2 FROM t) THEN + | SELECT 42; + | WHEN (SELECT COUNT(*) > 1 FROM t) THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END CASE; + |END + |""".stripMargin + + val expected = Seq(Seq.empty[Row], Seq.empty[Row], Seq.empty[Row], Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + + test("searched case no cases matched no else") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 = 2 THEN + | SELECT 42; + | WHEN 1 = 3 THEN + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq() + verifySqlScriptResult(commands, expected) + } + + test("searched case when evaluates to null") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a BOOLEAN) USING parquet; + | CASE + | WHEN (SELECT * FROM t) THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + + checkError( + exception = intercept[SqlScriptingException] ( + runSqlScript(commands) + ), + condition = "BOOLEAN_STATEMENT_WITH_EMPTY_ROW", + parameters = Map("invalidStatement" -> "(SELECT * FROM T)") + ) + } + } + + test("searched case with non boolean condition - constant") { + val commands = + """ + |BEGIN + | CASE + | WHEN 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + + checkError( + exception = intercept[SqlScriptingException] ( + runSqlScript(commands) + ), + condition = "INVALID_BOOLEAN_STATEMENT", + parameters = Map("invalidStatement" -> "1") + ) + } + + test("searched case with too many rows in subquery condition") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a BOOLEAN) USING parquet; + | INSERT INTO t VALUES (true); + | INSERT INTO t VALUES (true); + | CASE + | WHEN (SELECT * FROM t) THEN + | SELECT 1; + | END CASE; + |END + |""".stripMargin + + checkError( + exception = intercept[SparkException] ( + runSqlScript(commands) + ), + condition = "SCALAR_SUBQUERY_TOO_MANY_ROWS", + parameters = Map.empty, + context = ExpectedContext(fragment = "(SELECT * FROM t)", start = 124, stop = 140) + ) + } + } + + test("simple case") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("simple case nested") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 1 THEN + | CASE 2 + | WHEN (SELECT 3) THEN + | SELECT 41; + | ELSE + | SELECT 42; + | END CASE; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("simple case second case") { + val commands = + """ + |BEGIN + | CASE (SELECT 2) + | WHEN 1 THEN + | SELECT 1; + | WHEN 2 THEN + | SELECT 42; + | WHEN (SELECT * FROM t) THEN + | SELECT * FROM b; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + + test("simple case going in else") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 2 THEN + | SELECT 1; + | WHEN 3 THEN + | SELECT 2; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq(Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + + test("simple case with count") { + withTable("t") { + val commands = + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |INSERT INTO t VALUES (1, 'a', 1.0); + |INSERT INTO t VALUES (1, 'a', 1.0); + |CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 41; + | WHEN 2 THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + + val expected = Seq(Seq.empty[Row], Seq.empty[Row], Seq.empty[Row], Seq(Row(42))) + verifySqlScriptResult(commands, expected) + } + } + + test("simple case else with count") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + | INSERT INTO t VALUES (1, 'a', 1.0); + | INSERT INTO t VALUES (2, 'b', 2.0); + | CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 42; + | WHEN 3 THEN + | SELECT 43; + | ELSE + | SELECT 44; + | END CASE; + |END + |""".stripMargin + + val expected = Seq(Seq.empty[Row], Seq.empty[Row], Seq.empty[Row], Seq(Row(44))) + verifySqlScriptResult(commands, expected) + } + } + + test("simple case no cases matched no else") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN 2 THEN + | SELECT 42; + | WHEN 3 THEN + | SELECT 43; + | END CASE; + |END + |""".stripMargin + val expected = Seq() + verifySqlScriptResult(commands, expected) + } + + test("simple case mismatched types") { + val commands = + """ + |BEGIN + | CASE 1 + | WHEN "one" THEN + | SELECT 42; + | END CASE; + |END + |""".stripMargin + + checkError( + exception = intercept[SparkNumberFormatException] ( + runSqlScript(commands) + ), + condition = "CAST_INVALID_INPUT", + parameters = Map( + "expression" -> "'one'", + "sourceType" -> "\"STRING\"", + "targetType" -> "\"BIGINT\""), + context = ExpectedContext(fragment = "\"one\"", start = 23, stop = 27) + ) + } + + test("simple case compare with null") { + withTable("t") { + val commands = + """ + |BEGIN + | CREATE TABLE t (a INT) USING parquet; + | CASE (SELECT COUNT(*) FROM t) + | WHEN 1 THEN + | SELECT 42; + | ELSE + | SELECT 43; + | END CASE; + |END + |""".stripMargin + + val expected = Seq(Seq.empty[Row], Seq(Row(43))) + verifySqlScriptResult(commands, expected) + } + } + test("if's condition must be a boolean statement") { withTable("t") { val commands = @@ -378,13 +755,16 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { | END IF; |END |""".stripMargin + val exception = intercept[SqlScriptingException] { + runSqlScript(commands) + } checkError( - exception = intercept[SqlScriptingException] ( - runSqlScript(commands) - ), + exception = exception, condition = "INVALID_BOOLEAN_STATEMENT", parameters = Map("invalidStatement" -> "1") ) + assert(exception.origin.line.isDefined) + assert(exception.origin.line.get == 3) } } @@ -400,13 +780,16 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { | END IF; |END |""".stripMargin + val exception = intercept[SqlScriptingException] { + runSqlScript(commands1) + } checkError( - exception = intercept[SqlScriptingException] ( - runSqlScript(commands1) - ), + exception = exception, condition = "BOOLEAN_STATEMENT_WITH_EMPTY_ROW", parameters = Map("invalidStatement" -> "(SELECT * FROM T1)") ) + assert(exception.origin.line.isDefined) + assert(exception.origin.line.get == 4) // too many rows ( > 1 ) val commands2 =