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


Reply via email to