This is an automated email from the ASF dual-hosted git repository. wenchen pushed a commit to branch branch-4.0 in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-4.0 by this push: new e2051ea00a49 [SPARK-51428][SQL] Assign Aliases for collated expression trees deterministically e2051ea00a49 is described below commit e2051ea00a49e7e7edb47ebfe3fae9c27b0b6ef2 Author: ilicmarkodb <marko.i...@databricks.com> AuthorDate: Wed Apr 30 10:40:50 2025 +0800 [SPARK-51428][SQL] Assign Aliases for collated expression trees deterministically ### What changes were proposed in this pull request? Fixed assigning aliases for collated expression trees deterministically. This problem was resolved earlier, but it is again present since introducing `DEFAULT COLLATION` clause for Views. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added golden file `view-with-default-collation.sql`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #50727 from ilicmarkodb/fix_aliases_names_and_attribute_ref_datatype. Authored-by: ilicmarkodb <marko.i...@databricks.com> Signed-off-by: Wenchen Fan <wenc...@databricks.com> (cherry picked from commit 0225903f6756603a007b70a0aecd09df096f1da7) Signed-off-by: Wenchen Fan <wenc...@databricks.com> --- .../spark/sql/catalyst/analysis/Analyzer.scala | 4 +- .../ApplyDefaultCollationToStringType.scala | 10 +- .../catalyst/analysis/CollationRulesRunner.scala | 29 + .../sql/catalyst/plans/logical/v2Commands.scala | 7 + .../apache/spark/sql/execution/command/views.scala | 7 +- .../view-with-default-collation.sql.out | 1250 ++++++++++++++++++++ .../inputs/view-with-default-collation.sql | 152 +++ .../results/view-with-default-collation.sql.out | 967 +++++++++++++++ .../sql/collation/DefaultCollationTestSuite.scala | 125 -- .../SparkExecuteStatementOperation.scala | 6 +- .../thriftserver/SparkGetColumnsOperation.scala | 2 +- .../thriftserver/ThriftServerQueryTestSuite.scala | 1 + 12 files changed, 2425 insertions(+), 135 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f50bbc79e70d..dc8dbe8eb90f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -529,7 +529,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor def apply(plan: LogicalPlan): LogicalPlan = { val collatedPlan = if (conf.getConf(SQLConf.RUN_COLLATION_TYPE_CASTS_BEFORE_ALIAS_ASSIGNMENT)) { - CollationTypeCasts(plan) + CollationRulesRunner(plan) } else { plan } @@ -2799,7 +2799,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor def apply(plan: LogicalPlan): LogicalPlan = { val collatedPlan = if (conf.getConf(SQLConf.RUN_COLLATION_TYPE_CASTS_BEFORE_ALIAS_ASSIGNMENT)) { - CollationTypeCasts(plan) + CollationRulesRunner(plan) } else { plan } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollationToStringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollationToStringType.scala index cea2988badf4..a238c44ce7f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollationToStringType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ApplyDefaultCollationToStringType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.{Cast, DefaultStringProducingExpression, Expression, Literal, SubqueryExpression} -import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, AlterColumnSpec, AlterTableCommand, AlterViewAs, ColumnDefinition, CreateTable, CreateView, LogicalPlan, QualifiedColType, ReplaceColumns, ReplaceTable, V2CreateTablePlan} +import org.apache.spark.sql.catalyst.plans.logical.{AddColumns, AlterColumns, AlterColumnSpec, AlterTableCommand, AlterViewAs, ColumnDefinition, CreateTable, CreateTempView, CreateView, LogicalPlan, QualifiedColType, ReplaceColumns, ReplaceTable, V2CreateTablePlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.types.{DataType, StringType} @@ -52,6 +52,11 @@ object ApplyDefaultCollationToStringType extends Rule[LogicalPlan] { case createView: CreateView => createView.collation + // Temporary views are created via CreateViewCommand, which we can't import here. + // Instead, we use the CreateTempView trait to access the collation. + case createTempView: CreateTempView => + createTempView.collation + case replaceTable: ReplaceTable => replaceTable.tableSpec.collation @@ -83,7 +88,8 @@ object ApplyDefaultCollationToStringType extends Rule[LogicalPlan] { private def isCreateOrAlterPlan(plan: LogicalPlan): Boolean = plan match { // For CREATE TABLE, only v2 CREATE TABLE command is supported. // Also, table DEFAULT COLLATION cannot be specified through CREATE TABLE AS SELECT command. - case _: V2CreateTablePlan | _: ReplaceTable | _: CreateView | _: AlterViewAs => true + case _: V2CreateTablePlan | _: ReplaceTable | _: CreateView | _: AlterViewAs | + _: CreateTempView => true case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationRulesRunner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationRulesRunner.scala new file mode 100644 index 000000000000..76a0ead1736d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationRulesRunner.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +object CollationRulesRunner { + def apply(plan: LogicalPlan): LogicalPlan = { + Seq( + ApplyDefaultCollationToStringType, + CollationTypeCasts + ).foldLeft(plan) { case (plan, rule) => rule(plan) } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 1056a30c5f75..f8c1b2a90145 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -1376,6 +1376,13 @@ case class CreateView( } } +/** + * Used to apply ApplyDefaultCollationToStringType to CreateViewCommand + */ +trait CreateTempView { + val collation: Option[String] +} + /** * The logical plan of the ALTER VIEW ... SET TBLPROPERTIES command. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index f6e7dde996e2..c73fedc3697e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.{SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, GlobalTempView, LocalTempView, SchemaEvolution, SchemaUnsupported, ViewSchemaMode, ViewType} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, TemporaryViewRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, SubqueryExpression, VariableReference} -import org.apache.spark.sql.catalyst.plans.logical.{AnalysisOnlyCommand, CTEInChildren, CTERelationDef, LogicalPlan, Project, View, WithCTE} +import org.apache.spark.sql.catalyst.plans.logical.{AnalysisOnlyCommand, CreateTempView, CTEInChildren, CTERelationDef, LogicalPlan, Project, View, WithCTE} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.classic.ClassicConversions.castToImpl import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper @@ -76,7 +76,10 @@ case class CreateViewCommand( viewSchemaMode: ViewSchemaMode = SchemaUnsupported, isAnalyzed: Boolean = false, referredTempFunctions: Seq[String] = Seq.empty) - extends RunnableCommand with AnalysisOnlyCommand with CTEInChildren { + extends RunnableCommand + with AnalysisOnlyCommand + with CTEInChildren + with CreateTempView { import ViewHelper._ diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out new file mode 100644 index 000000000000..697208e9c931 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/view-with-default-collation.sql.out @@ -0,0 +1,1250 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a' +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, UNICODE, SELECT 'a', false, false, PersistedView, COMPENSATION, true + +- Project [a AS 'a' collate UNICODE#x] + +- OneRowRelation + + +-- !query +SELECT * FROM v +-- !query analysis +Project ['a' collate UNICODE#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, ['a' collate UNICODE#x]) + +- Project [cast('a' collate UNICODE#x as string collate UNICODE) AS 'a' collate UNICODE#x] + +- Project [a AS 'a' collate UNICODE#x] + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1 +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, UTF8_LCASE, SELECT 'a' AS c1, false, false, PersistedView, COMPENSATION, true + +- Project [a AS c1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x] + +- Project [a AS c1#x] + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('a'), ('A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č' +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, sr_CI_AI, SELECT c1 FROM t WHERE 'ć' = 'č', false, false, PersistedView, COMPENSATION, true + +- Project [c1#x] + +- Filter (ć = č) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x] + +- Project [c1#x] + +- Filter (ć = č) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x] + +- Project [c1#x] + +- Filter (ć = č) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, UNICODE, SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t, false, false, PersistedView, COMPENSATION, true + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query analysis +Distinct ++- Project [collation(c1#x) AS collation(c1)#x] + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = c) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, UNICODE, SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t, false, false, PersistedView, COMPENSATION, true + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query analysis +Distinct ++- Project [collation(c1#x) AS collation(c1)#x] + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = c) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c' +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, UTF8_LCASE, SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c', false, false, PersistedView, COMPENSATION, true + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c2#x = a) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'b' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c2#x = b) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT * FROM v +-- !query analysis +Project [c1#x, c2#x, 'c' collate UTF8_LCASE#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1, c2] ++- Project [cast(col1#x as string) AS c1#x, cast(col2#x as string collate UTF8_LCASE) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, sr_CI_AI, SELECT *, 'ć' AS c3 FROM t, false, true, PersistedView, COMPENSATION, true + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query analysis +Distinct ++- Project [collation(c1#x) AS collation(c1)#x] + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c2) FROM v +-- !query analysis +Distinct ++- Project [collation(c2#x) AS collation(c2)#x] + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c3) FROM v +-- !query analysis +Distinct ++- Project [collation(c3#x) AS collation(c3)#x] + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c2#x = a) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c3 = 'Č' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c3#x = Č) + +- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1 +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, UTF8_LCASE, SELECT 1, false, false, PersistedView, COMPENSATION, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 +-- !query analysis +AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT 'a' AS c1, 'b' AS c2, true + +- Project [a AS c1#x, b AS c2#x] + +- OneRowRelation + + +-- !query +SELECT COLLATION(c1) FROM v +-- !query analysis +Project [collation(c1#x) AS collation(c1)#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x] + +- Project [a AS c1#x, b AS c2#x] + +- OneRowRelation + + +-- !query +SELECT COLLATION(c2) FROM v +-- !query analysis +Project [collation(c2#x) AS collation(c2)#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x] + +- Project [a AS c1#x, b AS c2#x] + +- OneRowRelation + + +-- !query +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' +-- !query analysis +AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT 'c' AS c3 WHERE 'a' = 'A', true + +- Project [c AS c3#x] + +- Filter (a = A) + +- OneRowRelation + + +-- !query +SELECT COLLATION(c3) FROM v +-- !query analysis +Project [collation(c3#x) AS collation(c3)#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c3#x]) + +- Project [cast(c3#x as string collate UTF8_LCASE) AS c3#x] + +- Project [c AS c3#x] + +- Filter (a = A) + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('a', 'b', 1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1, c2, c3] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x, cast(col2#x as string) AS c2#x, cast(col3#x as int) AS c3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +CREATE VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1 +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`v`, sr_CI_AI, SELECT 'a' AS c1, false, false, PersistedView, COMPENSATION, true + +- Project [a AS c1#x] + +- OneRowRelation + + +-- !query +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č' +-- !query analysis +AlterViewAsCommand `spark_catalog`.`default`.`v`, SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č', true + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +SELECT COLLATION(c4) FROM v +-- !query analysis +Project [collation(c4#x) AS collation(c4)#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x, c4#x, c5#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string) AS c2#x, cast(c3#x as int) AS c3#x, cast(c4#x as string collate sr_CI_AI) AS c4#x, cast(c5#x as string collate sr_CI_AI) AS c5#x] + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +SELECT COLLATION(c5) FROM v +-- !query analysis +Project [collation(c5#x) AS collation(c5)#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x, c4#x, c5#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string) AS c2#x, cast(c3#x as int) AS c3#x, cast(c4#x as string collate sr_CI_AI) AS c4#x, cast(c5#x as string collate sr_CI_AI) AS c5#x] + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +SELECT c5 FROM v +-- !query analysis +Project [c5#x] ++- SubqueryAlias spark_catalog.default.v + +- View (`spark_catalog`.`default`.`v`, [c1#x, c2#x, c3#x, c4#x, c5#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string) AS c2#x, cast(c3#x as int) AS c3#x, cast(c4#x as string collate sr_CI_AI) AS c4#x, cast(c5#x as string collate sr_CI_AI) AS c5#x] + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`v`, true, true, false + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a' +-- !query analysis +CreateViewCommand `v`, UNICODE, SELECT 'a', false, false, LocalTempView, UNSUPPORTED, true + +- Project [a AS 'a' collate UNICODE#x] + +- OneRowRelation + + +-- !query +SELECT * FROM v +-- !query analysis +Project ['a' collate UNICODE#x] ++- SubqueryAlias v + +- View (`v`, ['a' collate UNICODE#x]) + +- Project [cast('a' collate UNICODE#x as string collate UNICODE) AS 'a' collate UNICODE#x] + +- Project [a AS 'a' collate UNICODE#x] + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1 +-- !query analysis +CreateViewCommand `v`, UTF8_LCASE, SELECT 'a' AS c1, false, false, LocalTempView, UNSUPPORTED, true + +- Project [a AS c1#x] + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x] + +- Project [a AS c1#x] + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('a'), ('A') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č' +-- !query analysis +CreateViewCommand `v`, sr_CI_AI, SELECT c1 FROM t WHERE 'ć' = 'č', false, false, LocalTempView, UNSUPPORTED, true + +- Project [c1#x] + +- Filter (ć = č) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x] + +- Project [c1#x] + +- Filter (ć = č) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x] + +- Project [c1#x] + +- Filter (ć = č) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query analysis +CreateViewCommand `v`, UNICODE, SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t, false, false, LocalTempView, UNSUPPORTED, true + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query analysis +Distinct ++- Project [collation(c1#x) AS collation(c1)#x] + +- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = c) + +- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x] + +- LocalRelation [col1#x] + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query analysis +CreateViewCommand `v`, UNICODE, SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t, false, false, LocalTempView, UNSUPPORTED, true + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query analysis +Distinct ++- Project [collation(c1#x) AS collation(c1)#x] + +- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = c) + +- SubqueryAlias v + +- View (`v`, [c1#x]) + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- Project [cast(c1#x as string collate sr_AI) AS c1#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c' +-- !query analysis +CreateViewCommand `v`, UTF8_LCASE, SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c', false, false, LocalTempView, UNSUPPORTED, true + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c2#x = a) + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'b' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c2#x = b) + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +SELECT * FROM v +-- !query analysis +Project [c1#x, c2#x, 'c' collate UTF8_LCASE#x] ++- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, 'c' collate UTF8_LCASE#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast('c' collate UTF8_LCASE#x as string collate UTF8_LCASE) AS 'c' collate UTF8_LCASE#x] + +- Project [a AS c1#x, scalar-subquery#x [] AS c2#x, c AS 'c' collate UTF8_LCASE#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((a = A) = true) THEN a ELSE b END AS CASE WHEN (('a' collate UTF8_LCASE = 'A' collate UTF8_LCASE) = true) THEN 'a' collate UTF8_LCASE ELSE 'b' collate UTF8_LCASE END#x] + : : +- OneRowRelation + : +- Filter (scalar-subquery#x [] = B) + : : +- Project [b AS 'b' collate UTF8_LCASE#x] + : : +- Filter (c = C) + : : +- OneRowRelation + : +- OneRowRelation + +- OneRowRelation + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b') +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1, c2] ++- Project [cast(col1#x as string) AS c1#x, cast(col2#x as string collate UTF8_LCASE) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t +-- !query analysis +CreateViewCommand `v`, sr_CI_AI, SELECT *, 'ć' AS c3 FROM t, false, true, LocalTempView, UNSUPPORTED, true + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query analysis +Distinct ++- Project [collation(c1#x) AS collation(c1)#x] + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c2) FROM v +-- !query analysis +Distinct ++- Project [collation(c2#x) AS collation(c2)#x] + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT DISTINCT COLLATION(c3) FROM v +-- !query analysis +Distinct ++- Project [collation(c3#x) AS collation(c3)#x] + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c1#x = A) + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c2#x = a) + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +SELECT COUNT(*) FROM v WHERE c3 = 'Č' +-- !query analysis +Aggregate [count(1) AS count(1)#xL] ++- Filter (c3#x = Č) + +- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x]) + +- Project [cast(c1#x as string) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x, cast(c3#x as string collate sr_CI_AI) AS c3#x] + +- Project [c1#x, c2#x, ć AS c3#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1 +-- !query analysis +CreateViewCommand `v`, UTF8_LCASE, SELECT 1, false, false, LocalTempView, UNSUPPORTED, true + +- Project [1 AS 1#x] + +- OneRowRelation + + +-- !query +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 +-- !query analysis +AlterViewAsCommand `v`, SELECT 'a' AS c1, 'b' AS c2, true + +- Project [a AS c1#x, b AS c2#x] + +- OneRowRelation + + +-- !query +SELECT COLLATION(c1) FROM v +-- !query analysis +Project [collation(c1#x) AS collation(c1)#x] ++- SubqueryAlias v + +- View (`v`, [c1#x, c2#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x] + +- Project [a AS c1#x, b AS c2#x] + +- OneRowRelation + + +-- !query +SELECT COLLATION(c2) FROM v +-- !query analysis +Project [collation(c2#x) AS collation(c2)#x] ++- SubqueryAlias v + +- View (`v`, [c1#x, c2#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string collate UTF8_LCASE) AS c2#x] + +- Project [a AS c1#x, b AS c2#x] + +- OneRowRelation + + +-- !query +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' +-- !query analysis +AlterViewAsCommand `v`, SELECT 'c' AS c3 WHERE 'a' = 'A', true + +- Project [c AS c3#x] + +- Filter (a = A) + +- OneRowRelation + + +-- !query +SELECT COLLATION(c3) FROM v +-- !query analysis +Project [collation(c3#x) AS collation(c3)#x] ++- SubqueryAlias v + +- View (`v`, [c3#x]) + +- Project [cast(c3#x as string collate UTF8_LCASE) AS c3#x] + +- Project [c AS c3#x] + +- Filter (a = A) + +- OneRowRelation + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT) +-- !query analysis +CreateDataSourceTableCommand `spark_catalog`.`default`.`t`, false + + +-- !query +INSERT INTO t VALUES ('a', 'b', 1) +-- !query analysis +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t], Append, `spark_catalog`.`default`.`t`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t), [c1, c2, c3] ++- Project [cast(col1#x as string collate UTF8_LCASE) AS c1#x, cast(col2#x as string) AS c2#x, cast(col3#x as int) AS c3#x] + +- LocalRelation [col1#x, col2#x, col3#x] + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1 +-- !query analysis +CreateViewCommand `v`, sr_CI_AI, SELECT 'a' AS c1, false, false, LocalTempView, UNSUPPORTED, true + +- Project [a AS c1#x] + +- OneRowRelation + + +-- !query +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č' +-- !query analysis +AlterViewAsCommand `v`, SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č', true + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +SELECT COLLATION(c4) FROM v +-- !query analysis +Project [collation(c4#x) AS collation(c4)#x] ++- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x, c4#x, c5#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string) AS c2#x, cast(c3#x as int) AS c3#x, cast(c4#x as string collate sr_CI_AI) AS c4#x, cast(c5#x as string collate sr_CI_AI) AS c5#x] + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +SELECT COLLATION(c5) FROM v +-- !query analysis +Project [collation(c5#x) AS collation(c5)#x] ++- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x, c4#x, c5#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string) AS c2#x, cast(c3#x as int) AS c3#x, cast(c4#x as string collate sr_CI_AI) AS c4#x, cast(c5#x as string collate sr_CI_AI) AS c5#x] + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +SELECT c5 FROM v +-- !query analysis +Project [c5#x] ++- SubqueryAlias v + +- View (`v`, [c1#x, c2#x, c3#x, c4#x, c5#x]) + +- Project [cast(c1#x as string collate UTF8_LCASE) AS c1#x, cast(c2#x as string) AS c2#x, cast(c3#x as int) AS c3#x, cast(c4#x as string collate sr_CI_AI) AS c4#x, cast(c5#x as string collate sr_CI_AI) AS c5#x] + +- Project [c1#x, c2#x, c3#x, c AS c4#x, scalar-subquery#x [] AS c5#x] + : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] + : : +- Project [CASE WHEN ((š = S) = true) THEN d ELSE b END AS CASE WHEN (('š' collate sr_CI_AI = 'S' collate sr_CI_AI) = true) THEN 'd' collate sr_CI_AI ELSE 'b' collate sr_CI_AI END#x] + : : +- OneRowRelation + : +- OneRowRelation + +- Filter ((c1#x = A) AND (ć = Č)) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[c1#x,c2#x,c3#x] parquet + + +-- !query +DROP VIEW IF EXISTS v +-- !query analysis +DropTempViewCommand v + + +-- !query +DROP TABLE IF EXISTS t +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t diff --git a/sql/core/src/test/resources/sql-tests/inputs/view-with-default-collation.sql b/sql/core/src/test/resources/sql-tests/inputs/view-with-default-collation.sql new file mode 100644 index 000000000000..765e5177f3c0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/view-with-default-collation.sql @@ -0,0 +1,152 @@ +-- CREATE VIEW with DEFAULT COLLATION +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a'; +SELECT * FROM v; + +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a'), ('A'); +CREATE VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č'; +SELECT COUNT(*) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c'; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c2 = 'b'; +SELECT * FROM v; + +-- CREATE OR REPLACE VIEW with DEFAULT COLLATION +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b'); +CREATE OR REPLACE VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT DISTINCT COLLATION(c2) FROM v; +SELECT DISTINCT COLLATION(c3) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c3 = 'Č'; + +-- ALTER VIEW check default collation +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1; +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2; +SELECT COLLATION(c1) FROM v; +SELECT COLLATION(c2) FROM v; +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A'; +SELECT COLLATION(c3) FROM v; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT); +INSERT INTO t VALUES ('a', 'b', 1); +CREATE VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1; +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č'; +SELECT COLLATION(c4) FROM v; +SELECT COLLATION(c5) FROM v; +SELECT c5 FROM v; + +-- CREATE TEMPORARY VIEW with DEFAULT COLLATION +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a'; +SELECT * FROM v; + +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a'), ('A'); +CREATE TEMPORARY VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č'; +SELECT COUNT(*) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c'; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c2 = 'b'; +SELECT * FROM v; + +-- CREATE OR REPLACE TEMPORARY VIEW with DEFAULT COLLATION +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b'); +CREATE OR REPLACE TEMPORARY VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT DISTINCT COLLATION(c2) FROM v; +SELECT DISTINCT COLLATION(c3) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c3 = 'Č'; + +-- ALTER TEMPORARY VIEW check default collation +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1; +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2; +SELECT COLLATION(c1) FROM v; +SELECT COLLATION(c2) FROM v; +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A'; +SELECT COLLATION(c3) FROM v; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT); +INSERT INTO t VALUES ('a', 'b', 1); +CREATE TEMPORARY VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1; +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č'; +SELECT COLLATION(c4) FROM v; +SELECT COLLATION(c5) FROM v; +SELECT c5 FROM v; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/sql/core/src/test/resources/sql-tests/results/view-with-default-collation.sql.out b/sql/core/src/test/resources/sql-tests/results/view-with-default-collation.sql.out new file mode 100644 index 000000000000..2078277d97a9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/view-with-default-collation.sql.out @@ -0,0 +1,967 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<'a' collate UNICODE:string collate UNICODE> +-- !query output +a + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a'), ('A') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'b' +-- !query schema +struct<count(1):bigint> +-- !query output +0 + + +-- !query +SELECT * FROM v +-- !query schema +struct<c1:string collate UTF8_LCASE,c2:string collate UTF8_LCASE,'c' collate UTF8_LCASE:string collate UTF8_LCASE> +-- !query output +a a c + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.UTF8_BINARY + + +-- !query +SELECT DISTINCT COLLATION(c2) FROM v +-- !query schema +struct<collation(c2):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT DISTINCT COLLATION(c3) FROM v +-- !query schema +struct<collation(c3):string> +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c3 = 'Č' +-- !query schema +struct<count(1):bigint> +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT COLLATION(c2) FROM v +-- !query schema +struct<collation(c2):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c3) FROM v +-- !query schema +struct<collation(c3):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'b', 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c4) FROM v +-- !query schema +struct<collation(c4):string> +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COLLATION(c5) FROM v +-- !query schema +struct<collation(c5):string> +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT c5 FROM v +-- !query schema +struct<c5:string collate sr_CI_AI> +-- !query output +d + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<'a' collate UNICODE:string collate UNICODE> +-- !query output +a + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a'), ('A') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'b' +-- !query schema +struct<count(1):bigint> +-- !query output +0 + + +-- !query +SELECT * FROM v +-- !query schema +struct<c1:string collate UTF8_LCASE,c2:string collate UTF8_LCASE,'c' collate UTF8_LCASE:string collate UTF8_LCASE> +-- !query output +a a c + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.UTF8_BINARY + + +-- !query +SELECT DISTINCT COLLATION(c2) FROM v +-- !query schema +struct<collation(c2):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT DISTINCT COLLATION(c3) FROM v +-- !query schema +struct<collation(c3):string> +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct<count(1):bigint> +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct<count(1):bigint> +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c3 = 'Č' +-- !query schema +struct<count(1):bigint> +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c1) FROM v +-- !query schema +struct<collation(c1):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT COLLATION(c2) FROM v +-- !query schema +struct<collation(c2):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c3) FROM v +-- !query schema +struct<collation(c3):string> +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'b', 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c4) FROM v +-- !query schema +struct<collation(c4):string> +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COLLATION(c5) FROM v +-- !query schema +struct<collation(c5):string> +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT c5 FROM v +-- !query schema +struct<c5:string collate sr_CI_AI> +-- !query output +d + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala index 5f7a7ff776ad..031af8200d7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala @@ -388,131 +388,6 @@ class DefaultCollationTestSuiteV1 extends DefaultCollationTestSuite { assertTableColumnCollation(testTable, "c4", "UTF8_BINARY") } } - - test("CREATE OR REPLACE VIEW with DEFAULT COLLATION") { - withTable(testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING, c2 STRING COLLATE UTF8_LCASE)") - sql(s"INSERT INTO $testTable VALUES ('a', 'a'), ('A', 'A'), ('b', 'b')") - withView(testView) { - // scalastyle:off - sql( - s"""CREATE OR REPLACE VIEW $testView - | DEFAULT COLLATION sr_ci_ai - | AS SELECT *, 'ć' AS c3 FROM $testTable - |""".stripMargin) - val prefix = "SYSTEM.BUILTIN" - checkAnswer(sql(s"SELECT DISTINCT COLLATION(c1) FROM $testView"), Row(s"$prefix.UTF8_BINARY")) - checkAnswer(sql(s"SELECT DISTINCT COLLATION(c2) FROM $testView"), Row(s"$prefix.UTF8_LCASE")) - checkAnswer(sql(s"SELECT DISTINCT COLLATION(c3) FROM $testView"), Row(s"$prefix.sr_CI_AI")) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Row(1)) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c2 = 'a'"), Row(2)) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c3 = 'Č'"), Row(3)) - // scalastyle:on - } - } - withView(testView) { - // scalastyle:off - sql( - s"""CREATE OR REPLACE VIEW $testView - | (c1) - | DEFAULT COLLATION sr_ai - | AS SELECT 'Ć' as c1 WHERE 'Ć' = 'C' - |""".stripMargin) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'Č'"), Row(1)) - // scalastyle:on - } - } - - test("CREATE VIEW with DEFAULT COLLATION") { - withView(testView) { - sql( - s"""CREATE VIEW $testView DEFAULT COLLATION UTF8_LCASE - | as SELECT 'a' as c1 - |""".stripMargin) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Seq(Row(1))) - } - withTable(testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE)") - sql(s"INSERT INTO $testTable VALUES ('a'), ('A')") - withView(testView) { - withSQLConf() { - // scalastyle:off - sql( - s"""CREATE VIEW $testView DEFAULT COLLATION SR_AI_CI - | AS SELECT c1 FROM $testTable - | WHERE 'ć' = 'č' - |""".stripMargin) - // scalastyle:on - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView"), Seq(Row(2))) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Seq(Row(2))) - } - } - } - withTable(testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE)") - // scalastyle:off - sql(s"INSERT INTO $testTable VALUES ('ć'), ('č')") - // scalastyle:on - withView(testView) { - sql( - s"""CREATE VIEW $testView DEFAULT COLLATION UNICODE - | AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM $testTable - |""".stripMargin) - val prefix = "SYSTEM.BUILTIN" - checkAnswer(sql(s"SELECT DISTINCT COLLATION(c1) FROM $testView"), Row(s"$prefix.sr_AI")) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'c'"), Row(2)) - } - } - withView(testView) { - sql( - s"""CREATE VIEW $testView DEFAULT COLLATION UTF8_LCASE - | AS SELECT 'a' AS c1, - | (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) - | WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2 - |""".stripMargin) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c1 = 'A'"), Seq(Row(1))) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c2 = 'a'"), Seq(Row(1))) - checkAnswer(sql(s"SELECT COUNT(*) FROM $testView WHERE c2 = 'b'"), Seq(Row(0))) - } - } - - test("ALTER VIEW check default collation") { - Seq("", "TEMPORARY").foreach { temporary => - withView(testView) { - sql(s"CREATE $temporary VIEW $testView DEFAULT COLLATION UTF8_LCASE AS SELECT 1") - sql(s"ALTER VIEW $testView AS SELECT 'a' AS c1, 'b' AS c2") - val prefix = "SYSTEM.BUILTIN" - checkAnswer(sql(s"SELECT COLLATION(c1) FROM $testView"), - Row(s"$prefix.UTF8_LCASE")) - checkAnswer(sql(s"SELECT COLLATION(c2) FROM $testView"), - Row(s"$prefix.UTF8_LCASE")) - sql(s"ALTER VIEW $testView AS SELECT 'c' AS c3 WHERE 'a' = 'A'") - checkAnswer(sql(s"SELECT COLLATION(c3) FROM $testView"), - Row(s"$prefix.UTF8_LCASE")) - } - withTable(testTable) { - sql(s"CREATE TABLE $testTable (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT)") - sql(s"INSERT INTO $testTable VALUES ('a', 'b', 1)") - withView(testView) { - sql(s"CREATE $temporary VIEW $testView DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1") - // scalastyle:off - sql( - s"""ALTER VIEW $testView AS - | SELECT *, 'c' AS c4, - | (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 - | FROM $testTable - | WHERE c1 = 'A' AND 'ć' = 'Č'""".stripMargin) - // scalastyle:on - val prefix = "SYSTEM.BUILTIN" - checkAnswer(sql(s"SELECT COLLATION(c4) FROM $testView"), - Row(s"$prefix.sr_CI_AI")) - checkAnswer(sql(s"SELECT COLLATION(c5) FROM $testView"), - Row(s"$prefix.sr_CI_AI")) - checkAnswer(sql(s"SELECT c5 FROM $testView"), Row("d")) - } - } - } - } } class DefaultCollationTestSuiteV2 extends DefaultCollationTestSuite with DatasourceV2SQLBase { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index caa85ebe57de..ce857ce8f866 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -339,7 +339,9 @@ object SparkExecuteStatementOperation { case LongType => TTypeId.BIGINT_TYPE case FloatType => TTypeId.FLOAT_TYPE case DoubleType => TTypeId.DOUBLE_TYPE - case StringType => TTypeId.STRING_TYPE + case _: CharType => TTypeId.CHAR_TYPE + case _: VarcharType => TTypeId.VARCHAR_TYPE + case _: StringType => TTypeId.STRING_TYPE case _: DecimalType => TTypeId.DECIMAL_TYPE case DateType => TTypeId.DATE_TYPE // TODO: Shall use TIMESTAMPLOCALTZ_TYPE, keep AS-IS now for @@ -353,8 +355,6 @@ object SparkExecuteStatementOperation { case _: ArrayType => TTypeId.ARRAY_TYPE case _: MapType => TTypeId.MAP_TYPE case _: StructType => TTypeId.STRUCT_TYPE - case _: CharType => TTypeId.CHAR_TYPE - case _: VarcharType => TTypeId.VARCHAR_TYPE case other => throw new IllegalArgumentException(s"Unrecognized type name: ${other.catalogString}") } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 1004ca8cf271..c0552f715442 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -184,9 +184,9 @@ private[hive] class SparkGetColumnsOperation( case FloatType => java.sql.Types.FLOAT case DoubleType => java.sql.Types.DOUBLE case _: DecimalType => java.sql.Types.DECIMAL - case StringType => java.sql.Types.VARCHAR case VarcharType(_) => java.sql.Types.VARCHAR case CharType(_) => java.sql.Types.CHAR + case _: StringType => java.sql.Types.VARCHAR case BinaryType => java.sql.Types.BINARY case DateType => java.sql.Types.DATE case TimestampType | TimestampNTZType => java.sql.Types.TIMESTAMP diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index bc367d0cc856..3fdb8fb7ab8a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -104,6 +104,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ // SPARK-42921 "timestampNTZ/datetime-special-ansi.sql", // SPARK-47264 + "view-with-default-collation.sql", "collations.sql", "listagg-collations.sql", "pipe-operators.sql", --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org